Merge remote-tracking branch 'upstream/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 78658c3..4e0bc83 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>
@@ -278,11 +273,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 6e4ffc9..5d70035 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>
@@ -197,7 +193,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>
@@ -210,7 +205,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>
@@ -356,7 +350,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>
@@ -371,7 +365,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..dd5b7d2 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";
@@ -88,9 +88,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 +114,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 7c1bbf0..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ /dev/null
@@ -1,751 +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");
-            scanner = new IsolatedScanner(scanner);
-          }
-          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 bfbad11..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ /dev/null
@@ -1,781 +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");
-            scanner = new IsolatedScanner(scanner);
-          }
-          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..9b61e53 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;
@@ -35,8 +36,10 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.metadata.AbstractTabletFile;
 import org.apache.accumulo.core.security.Authorizations;
 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 = AbstractTabletFile.requireRowRange(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..79ea0bd 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;
@@ -211,10 +212,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 +351,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 +389,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 +407,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/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 d02a2c7..acc0602 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
@@ -21,7 +21,6 @@
 import static com.google.common.base.Preconditions.checkArgument;
 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;
@@ -36,6 +35,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;
@@ -75,12 +75,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;
@@ -147,8 +149,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());
@@ -851,18 +839,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 e980230..02aeef6 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;
@@ -50,6 +50,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 +64,19 @@
 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;
@@ -88,6 +87,8 @@
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
 
+import com.google.common.net.HostAndPort;
+
 class ConditionalWriterImpl implements ConditionalWriter {
 
   private static final int MAX_SLEEP = 30000;
@@ -147,7 +148,7 @@
         count--;
         return result;
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
 
@@ -226,7 +227,7 @@
 
     @Override
     public void run() {
-      TabletClientService.Iface client = null;
+      TabletIngestClientService.Iface client = null;
 
       for (SessionID sid : sessions) {
         if (!sid.isActive()) {
@@ -478,7 +479,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) {
@@ -547,18 +548,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();
 
@@ -710,7 +712,7 @@
   }
 
   private void invalidateSession(long sessionId, HostAndPort location) throws TException {
-    TabletClientService.Iface client = null;
+    TabletIngestClientService.Iface client = null;
 
     TInfo tinfo = TraceUtil.traceInfo();
 
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 8f30ded..040a765 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;
@@ -71,13 +69,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;
@@ -86,6 +83,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);
@@ -158,17 +157,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);
     }
   }
 
@@ -194,7 +191,7 @@
           if (fatalException instanceof RuntimeException) {
             throw (RuntimeException) fatalException;
           } else {
-            throw new RuntimeException(fatalException);
+            throw new IllegalStateException(fatalException);
           }
         }
 
@@ -205,13 +202,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);
       }
     }
   }
@@ -251,9 +249,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) {
 
@@ -284,7 +282,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 604aece..f36944d 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;
@@ -42,6 +42,7 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 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;
@@ -65,15 +66,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.hadoop.io.Text;
 import org.apache.thrift.TApplicationException;
@@ -81,6 +80,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;
 
@@ -100,7 +101,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()) {
@@ -276,7 +276,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)
@@ -331,18 +331,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);
               }
             }
@@ -368,7 +368,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);
         } catch (AccumuloSecurityException e) {
@@ -379,7 +379,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";
@@ -394,7 +394,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
@@ -469,7 +469,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)) {
@@ -521,17 +521,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();
@@ -563,19 +563,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()) {
@@ -588,7 +587,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());
@@ -613,7 +612,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);
 
@@ -628,23 +627,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,
@@ -652,7 +652,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;
@@ -666,7 +666,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()) {
@@ -691,7 +691,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) {
@@ -702,8 +702,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) {
@@ -750,7 +750,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 a3d38aa..b3f205f 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;
@@ -42,7 +42,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;
@@ -52,13 +51,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 fc5a52f..463b37c 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
@@ -294,6 +291,29 @@
   GENERAL_DELEGATION_TOKEN_UPDATE_INTERVAL("general.delegation.token.update.interval", "1d",
       PropertyType.TIMEDURATION, "The length of time between generation of new secret keys.",
       "1.7.0"),
+  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.",
@@ -311,19 +331,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",
@@ -335,28 +348,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,
@@ -377,11 +373,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"),
@@ -389,29 +380,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",
@@ -428,6 +402,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,
@@ -445,6 +428,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 refreshed.",
       "2.1.0"),
@@ -522,6 +512,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"
@@ -535,29 +527,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"
@@ -565,35 +540,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,
@@ -651,19 +605,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}]"
@@ -671,17 +633,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}]"
@@ -689,20 +657,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",
@@ -711,29 +685,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"),
@@ -741,12 +696,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"
@@ -754,23 +703,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,
@@ -806,16 +738,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"
@@ -824,7 +748,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"
@@ -836,17 +761,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"),
@@ -874,6 +788,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"
@@ -882,9 +797,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.",
@@ -902,17 +818,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,
@@ -954,7 +859,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"),
@@ -967,41 +872,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,"
@@ -1030,23 +900,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 "
@@ -1066,10 +923,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,
@@ -1077,15 +937,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,
@@ -1254,31 +1105,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"
@@ -1319,122 +1149,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,
@@ -1469,6 +1183,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,
@@ -1517,54 +1234,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;
@@ -1721,6 +1391,8 @@
     ReplacedBy rb = getAnnotation(ReplacedBy.class);
     if (rb != null) {
       replacedBy = rb.property();
+    } else {
+      isReplaced = false;
     }
     annotationsComputed = true;
   }
@@ -1819,8 +1491,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())
@@ -1880,15 +1550,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..1fe82e4 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
@@ -24,7 +24,6 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hadoop.io.Text;
 
@@ -105,10 +104,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/dataImpl/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/dataImpl/KeyExtent.java
index c0a40dd..434c0f7 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
@@ -46,8 +46,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;
@@ -476,11 +475,11 @@
   }
 
   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 +487,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 87d006c..858e6e6 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
@@ -37,11 +37,11 @@
 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;
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 8dadac9..1a14418 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;
@@ -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);
 
@@ -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);
         }
       }
     }
@@ -220,24 +221,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
@@ -262,7 +255,7 @@
         }
       }
     }, 3, SECONDS));
-    executor = pool;
+    this.executor = pool;
   }
 
   // get a transaction id back to the requester before doing any work
@@ -283,7 +276,7 @@
             store.push(tid, repo);
           } catch (StackOverflowException e) {
             // this should not happen
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
 
@@ -400,7 +393,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/ZooStore.java b/core/src/main/java/org/apache/accumulo/core/fate/ZooStore.java
index ff7a297..b7f5539 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;
@@ -43,6 +45,7 @@
 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 +65,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 +76,7 @@
       oos.writeObject(o);
       return baos.toByteArray();
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -85,8 +87,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 +122,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 +165,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 +192,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 +204,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 +228,7 @@
           try {
             this.wait(1000);
           } catch (InterruptedException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
 
@@ -271,9 +274,9 @@
 
   @Override
   public void unreserve(long tid, long deferTime, TimeUnit deferTimeUnit) {
-    deferTime = TimeUnit.NANOSECONDS.convert(deferTime, deferTimeUnit);
+    Duration deferDuration = Duration.of(deferTime, deferTimeUnit.toChronoUnit());
 
-    if (deferTime < 0) {
+    if (deferDuration.compareTo(Duration.ZERO) < 0) {
       throw new IllegalArgumentException("deferTime < 0 : " + deferTime);
     }
 
@@ -284,7 +287,7 @@
       }
 
       if (deferTime > 0) {
-        deferred.put(tid, System.nanoTime() + deferTime);
+        deferred.put(tid, NanoTime.nowPlus(deferDuration));
       }
 
       this.notifyAll();
@@ -317,7 +320,7 @@
             return null;
           }
         } catch (KeeperException.NoNodeException ex) {
-          throw new RuntimeException(ex);
+          throw new IllegalStateException(ex);
         }
 
         byte[] ser = zk.getData(txpath + "/" + top);
@@ -328,8 +331,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 +372,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 +388,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 +398,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 +428,7 @@
           try {
             this.wait(5000);
           } catch (InterruptedException e) {
-            throw new RuntimeException(e);
+            throw new IllegalStateException(e);
           }
         }
       }
@@ -439,8 +442,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 +458,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 +479,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 +502,8 @@
       }
     } catch (NoNodeException nne) {
       return null;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -513,8 +516,8 @@
         l.add(parseTid(txid));
       }
       return l;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -541,7 +544,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 +564,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 318f87a..78200ee 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;
@@ -49,11 +49,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;
@@ -66,7 +68,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;
@@ -216,7 +217,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;
         }
@@ -385,13 +386,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
@@ -454,7 +455,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);
@@ -474,8 +475,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();
 
@@ -497,8 +498,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));
 
@@ -506,7 +507,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 32ee710..65c0c0b 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,15 +21,15 @@
 import java.io.DataInputStream;
 import java.io.IOException;
 
-import org.apache.accumulo.core.data.Key;
 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 0183ebe..2167fdc 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 {
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 c7da8f4..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java
+++ /dev/null
@@ -1,186 +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.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 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 b11d100..d3a794a 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
@@ -37,6 +37,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 {
 
@@ -80,12 +81,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 11c5b40..c99d137 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;
@@ -54,6 +56,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;
@@ -71,16 +74,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 {
 
@@ -1033,8 +1040,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
           }
 
@@ -1100,16 +1107,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
@@ -1160,7 +1167,7 @@
     }
   }
 
-  public static class Reader extends HeapIterator implements FileSKVIterator {
+  public static class Reader extends HeapIterator implements RFileSKVIterator {
 
     private final CachableBlockFile.Reader reader;
 
@@ -1295,7 +1302,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) {
@@ -1308,7 +1315,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();
@@ -1334,47 +1341,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
@@ -1387,7 +1394,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) {
@@ -1476,6 +1483,7 @@
       return (lgCache == null ? 0 : lgCache.getNumLGSeeked());
     }
 
+    @Override
     public FileSKVIterator getIndex() throws IOException {
 
       ArrayList<Iterator<IndexEntry>> indexes = new ArrayList<>();
@@ -1488,7 +1496,7 @@
     }
 
     @Override
-    public FileSKVIterator getSample(SamplerConfigurationImpl sampleConfig) {
+    public Reader getSample(SamplerConfigurationImpl sampleConfig) {
       requireNonNull(sampleConfig);
 
       if (this.samplerConfig != null && this.samplerConfig.equals(sampleConfig)) {
@@ -1534,11 +1542,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);
@@ -1555,5 +1564,230 @@
     public void setCacheProvider(CacheProvider cacheProvider) {
       reader.setCacheProvider(cacheProvider);
     }
+
+    @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 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 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..be8d633 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
@@ -176,8 +176,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));
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 09c087a..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
@@ -228,7 +228,7 @@
       }
     } catch (ReflectiveOperationException e) {
       log.error(e.toString());
-      throw new RuntimeException(e);
+      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 bc3d317..0000000
--- a/core/src/main/java/org/apache/accumulo/core/iteratorsImpl/system/MapFileIterator.java
+++ /dev/null
@@ -1,121 +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.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.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 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 44d0d60..e1b9fe3 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
@@ -35,6 +35,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 {
 
@@ -116,12 +117,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/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/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..ae0d46f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
@@ -0,0 +1,106 @@
+/*
+ * 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.Objects;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * 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 = requireRowRange(range);
+  }
+
+  @Override
+  public Path getPath() {
+    return path;
+  }
+
+  @Override
+  public Range getRange() {
+    return range;
+  }
+
+  @Override
+  public boolean hasRange() {
+    return !range.isInfiniteStartKey() || !range.isInfiniteStopKey();
+  }
+
+  public static Range requireRowRange(Range range) {
+    if (!range.isInfiniteStartKey()) {
+      Preconditions.checkArgument(range.isStartKeyInclusive() && isOnlyRowSet(range.getStartKey()),
+          "Range is not a row range %s", range);
+    }
+
+    if (!range.isInfiniteStopKey()) {
+      Preconditions.checkArgument(!range.isEndKeyInclusive() && isOnlyRowSet(range.getEndKey())
+          && isExclusiveKey(range.getEndKey()), "Range is not a row range %s", range);
+    }
+
+    return range;
+  }
+
+  private static boolean isOnlyRowSet(Key key) {
+    return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0
+        && key.getColumnVisibilityData().length() == 0 && key.getTimestamp() == Long.MAX_VALUE;
+  }
+
+  private static boolean isExclusiveKey(Key key) {
+    var row = key.getRowData();
+    return row.length() > 0 && row.byteAt(row.length() - 1) == (byte) 0x00;
+  }
+
+  private static String stripZeroTail(ByteSequence row) {
+    if (row.byteAt(row.length() - 1) == (byte) 0x00) {
+      return row.subSequence(0, row.length() - 1).toString();
+    }
+    return row.toString();
+  }
+
+  @Override
+  public String toMinimalString() {
+    if (hasRange()) {
+      String startRow =
+          range.isInfiniteStartKey() ? "-inf" : stripZeroTail(range.getStartKey().getRowData());
+      String endRow =
+          range.isInfiniteStopKey() ? "+inf" : stripZeroTail(range.getEndKey().getRowData());
+      return getFileName() + " (" + startRow + "," + endRow + "]";
+    } else {
+      return getFileName();
+    }
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
similarity index 67%
rename from core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
rename to core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
index b2a9a5d..14b8b0c 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AccumuloTable.java
@@ -21,7 +21,26 @@
 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";
+/**
+ * Defines the name and id of all tables in the accumulo table namespace.
+ */
+public enum AccumuloTable {
+
+  ROOT("root", "+r"), METADATA("metadata", "!0");
+
+  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);
+  }
 }
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/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/ScanServerRefTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java
index ad169ca..291343e 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
@@ -26,7 +26,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
-public class ScanServerRefTabletFile extends TabletFile {
+public class ScanServerRefTabletFile extends ReferencedTabletFile {
 
   private final Value NULL_VALUE = new Value(new byte[0]);
   private final Text colf;
@@ -45,7 +45,7 @@
   }
 
   public String getRowSuffix() {
-    return this.getPathStr();
+    return this.getNormalizedPathStr();
   }
 
   public Text getServerAddress() {
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..16bb045 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,25 @@
  */
 package org.apache.accumulo.core.metadata;
 
+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 +49,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 +81,216 @@
    * 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
+    requireRowRange(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)
+    //
+    // 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) {
+    requireRowRange(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..931e415 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
@@ -26,18 +26,18 @@
 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.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
 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 +74,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 +106,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 +241,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 +268,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 +292,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();
 
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 b1b306b..0fd9912 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);
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..f2ad719 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;
@@ -47,16 +46,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 +62,82 @@
 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 SortedMap<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(ImmutableSortedMap.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 +159,8 @@
     COMPACT_ID,
     SPLIT_RATIO,
     SUSPEND,
-    CHOPPED,
-    ECOMP
+    ECOMP,
+    MERGED
   }
 
   public static class Location {
@@ -233,10 +263,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 +312,7 @@
     return location != null && location.getType() == LocationType.CURRENT;
   }
 
-  public Map<TabletFile,Long> getLoaded() {
+  public Map<StoredTabletFile,Long> getLoaded() {
     ensureFetched(ColumnType.LOADED);
     return loadedFiles;
   }
@@ -350,9 +377,9 @@
     return splitRatio;
   }
 
-  public boolean hasChopped() {
-    ensureFetched(ColumnType.CHOPPED);
-    return chopped;
+  public boolean hasMerged() {
+    ensureFetched(ColumnType.MERGED);
+    return merged;
   }
 
   public SortedMap<Key,Value> getKeyValues() {
@@ -375,7 +402,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 +419,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 +430,14 @@
       final String qual = key.getColumnQualifierData().toString();
 
       if (buildKeyValueMap) {
-        kvBuilder.put(key, kv.getValue());
+        tmBuilder.keyValue(key, kv.getValue());
       }
 
       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 +447,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 +464,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 +553,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 ImmutableSortedMap.Builder<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(Key key, Value value) {
+      if (this.keyValues == null) {
+        this.keyValues = ImmutableSortedMap.naturalOrder();
+      }
+      this.keyValues.put(key, value);
+    }
+
+    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 ddc9278..abfdfc9 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
@@ -47,6 +47,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>
@@ -457,6 +464,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>
@@ -472,40 +486,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) -->
@@ -581,6 +566,7 @@
 
   Logger LOG = LoggerFactory.getLogger(MetricsProducer.class);
 
+  String METRICS_LOW_MEMORY = "accumulo.detected.low.memory";
   String METRICS_COMPACTOR_PREFIX = "accumulo.compactor.";
   String METRICS_COMPACTOR_MAJC_STUCK = METRICS_COMPACTOR_PREFIX + "majc.stuck";
 
@@ -610,16 +596,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";
@@ -635,6 +617,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";
 
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/ColumnVisibility.java b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
index 94772fe..4b8784b 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
@@ -91,7 +91,7 @@
   /**
    * The node types in a parse tree for a visibility expression.
    */
-  public static enum NodeType {
+  public enum NodeType {
     EMPTY, TERM, OR, AND,
   }
 
@@ -151,7 +151,7 @@
 
     public ByteSequence getTerm(byte[] expression) {
       if (type != NodeType.TERM) {
-        throw new RuntimeException();
+        throw new IllegalStateException();
       }
 
       if (expression[start] == '"') {
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 bba5cbf..f7dd384 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.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -59,9 +59,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.HashMultimap;
 import com.google.common.collect.Multimap;
 
@@ -93,7 +92,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);
@@ -262,7 +260,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(
@@ -322,13 +320,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);
   }
@@ -422,7 +415,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/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/ConfigurableScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelector.java
index 2e79218..cf3c530 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;
@@ -41,7 +42,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;
@@ -149,7 +149,6 @@
  */
 public class ConfigurableScanServerSelector implements ScanServerSelector {
 
-  private 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'},"
@@ -263,9 +262,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;
@@ -365,8 +363,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 541ff7a..063c762 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
@@ -24,6 +24,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;
@@ -43,7 +44,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 2c83986..d9bca52 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
@@ -45,7 +45,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/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/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/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 a974836..0046af7 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;
@@ -35,16 +34,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;
@@ -52,6 +51,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class ExternalCompactionUtil {
 
   private static class RunningCompactionFuture {
@@ -100,25 +101,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) {
@@ -138,10 +135,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);
     }
   }
 
@@ -241,7 +238,7 @@
           results.add(new RunningCompaction(job, compactorAddress, rcf.getQueue()));
         }
       } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     });
     return results;
@@ -269,7 +266,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 6345f9d..6ea16a7 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,8 +36,8 @@
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 
-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 {
 
@@ -46,7 +45,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);
@@ -112,11 +111,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) {
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 31b2713..f3756c9 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 org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob 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 org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob 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<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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 org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob 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 org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob 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<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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<org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob> 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<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;
       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<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 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<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;
@@ -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);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public closeConditionalUpdate_args(closeConditionalUpdate_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 closeConditionalUpdate_args deepCopy() {
+      return new closeConditionalUpdate_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 closeConditionalUpdate_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 closeConditionalUpdate_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 closeConditionalUpdate_args)
+        return this.equals((closeConditionalUpdate_args)that);
+      return false;
+    }
+
+    public boolean equals(closeConditionalUpdate_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(closeConditionalUpdate_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("closeConditionalUpdate_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 closeConditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public closeConditionalUpdate_argsStandardScheme getScheme() {
+        return new closeConditionalUpdate_argsStandardScheme();
+      }
+    }
+
+    private static class closeConditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeConditionalUpdate_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, closeConditionalUpdate_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, closeConditionalUpdate_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 closeConditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public closeConditionalUpdate_argsTupleScheme getScheme() {
+        return new closeConditionalUpdate_argsTupleScheme();
+      }
+    }
+
+    private static class closeConditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeConditionalUpdate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, closeConditionalUpdate_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, closeConditionalUpdate_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 loadFiles_args implements org.apache.thrift.TBase<loadFiles_args, loadFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<loadFiles_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("loadFiles_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 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 DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("dir", 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.MAP, (short)7);
+    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)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new loadFiles_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new loadFiles_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 long tid; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String dir; // required
+    public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> files; // 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)2, "credentials"),
+      TID((short)3, "tid"),
+      DIR((short)4, "dir"),
+      FILES((short)7, "files"),
+      SET_TIME((short)6, "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 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // TID
+            return TID;
+          case 4: // DIR
+            return DIR;
+          case 7: // FILES
+            return FILES;
+          case 6: // 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.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.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.DIR, new org.apache.thrift.meta_data.FieldMetaData("dir", 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.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class), 
+              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, DataFileInfo.class)))));
+      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(loadFiles_args.class, metaDataMap);
+    }
+
+    public loadFiles_args() {
+    }
+
+    public loadFiles_args(
+      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)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.tid = tid;
+      setTidIsSet(true);
+      this.dir = dir;
+      this.files = files;
+      this.setTime = setTime;
+      setSetTimeIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public loadFiles_args(loadFiles_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);
+      }
+      this.tid = other.tid;
+      if (other.isSetDir()) {
+        this.dir = other.dir;
+      }
+      if (other.isSetFiles()) {
+        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> __this__files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>(other.files.size());
+        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,DataFileInfo>> other_element : other.files.entrySet()) {
+
+          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
+          java.util.Map<java.lang.String,DataFileInfo> other_element_value = other_element.getValue();
+
+          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__files_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
+
+          java.util.Map<java.lang.String,DataFileInfo> __this__files_copy_value = new java.util.HashMap<java.lang.String,DataFileInfo>(other_element_value.size());
+          for (java.util.Map.Entry<java.lang.String, DataFileInfo> other_element_value_element : other_element_value.entrySet()) {
+
+            java.lang.String other_element_value_element_key = other_element_value_element.getKey();
+            DataFileInfo other_element_value_element_value = other_element_value_element.getValue();
+
+            java.lang.String __this__files_copy_value_copy_key = other_element_value_element_key;
+
+            DataFileInfo __this__files_copy_value_copy_value = new DataFileInfo(other_element_value_element_value);
+
+            __this__files_copy_value.put(__this__files_copy_value_copy_key, __this__files_copy_value_copy_value);
+          }
+
+          __this__files.put(__this__files_copy_key, __this__files_copy_value);
+        }
+        this.files = __this__files;
+      }
+      this.setTime = other.setTime;
+    }
+
+    @Override
+    public loadFiles_args deepCopy() {
+      return new loadFiles_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      setTidIsSet(false);
+      this.tid = 0;
+      this.dir = null;
+      this.files = null;
+      setSetTimeIsSet(false);
+      this.setTime = false;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public loadFiles_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 loadFiles_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 loadFiles_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 getDir() {
+      return this.dir;
+    }
+
+    public loadFiles_args setDir(@org.apache.thrift.annotation.Nullable java.lang.String dir) {
+      this.dir = dir;
+      return this;
+    }
+
+    public void unsetDir() {
+      this.dir = null;
+    }
+
+    /** Returns true if field dir is set (has been assigned a value) and false otherwise */
+    public boolean isSetDir() {
+      return this.dir != null;
+    }
+
+    public void setDirIsSet(boolean value) {
+      if (!value) {
+        this.dir = null;
+      }
+    }
+
+    public int getFilesSize() {
+      return (this.files == null) ? 0 : this.files.size();
+    }
+
+    public void putToFiles(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.Map<java.lang.String,DataFileInfo> val) {
+      if (this.files == null) {
+        this.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>();
+      }
+      this.files.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> getFiles() {
+      return this.files;
+    }
+
+    public loadFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>> 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;
+      }
+    }
+
+    public boolean isSetTime() {
+      return this.setTime;
+    }
+
+    public loadFiles_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.clientImpl.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 DIR:
+        if (value == null) {
+          unsetDir();
+        } else {
+          setDir((java.lang.String)value);
+        }
+        break;
+
+      case FILES:
+        if (value == null) {
+          unsetFiles();
+        } else {
+          setFiles((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>)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 DIR:
+        return getDir();
+
+      case FILES:
+        return getFiles();
+
+      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 DIR:
+        return isSetDir();
+      case FILES:
+        return isSetFiles();
+      case SET_TIME:
+        return isSetSetTime();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof loadFiles_args)
+        return this.equals((loadFiles_args)that);
+      return false;
+    }
+
+    public boolean equals(loadFiles_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_dir = true && this.isSetDir();
+      boolean that_present_dir = true && that.isSetDir();
+      if (this_present_dir || that_present_dir) {
+        if (!(this_present_dir && that_present_dir))
+          return false;
+        if (!this.dir.equals(that.dir))
+          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_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 + ((isSetDir()) ? 131071 : 524287);
+      if (isSetDir())
+        hashCode = hashCode * 8191 + dir.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
+      if (isSetFiles())
+        hashCode = hashCode * 8191 + files.hashCode();
+
+      hashCode = hashCode * 8191 + ((setTime) ? 131071 : 524287);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(loadFiles_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(isSetDir(), other.isSetDir());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetDir()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dir, other.dir);
+        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(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("loadFiles_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("dir:");
+      if (this.dir == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.dir);
+      }
+      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("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 loadFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public loadFiles_argsStandardScheme getScheme() {
+        return new loadFiles_argsStandardScheme();
+      }
+    }
+
+    private static class loadFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<loadFiles_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, loadFiles_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: // 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: // DIR
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.dir = iprot.readString();
+                struct.setDirIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 7: // FILES
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map58 = iprot.readMapBegin();
+                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>(2*_map58.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key59;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,DataFileInfo> _val60;
+                  for (int _i61 = 0; _i61 < _map58.size; ++_i61)
+                  {
+                    _key59 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                    _key59.read(iprot);
+                    {
+                      org.apache.thrift.protocol.TMap _map62 = iprot.readMapBegin();
+                      _val60 = new java.util.HashMap<java.lang.String,DataFileInfo>(2*_map62.size);
+                      @org.apache.thrift.annotation.Nullable java.lang.String _key63;
+                      @org.apache.thrift.annotation.Nullable DataFileInfo _val64;
+                      for (int _i65 = 0; _i65 < _map62.size; ++_i65)
+                      {
+                        _key63 = iprot.readString();
+                        _val64 = new DataFileInfo();
+                        _val64.read(iprot);
+                        _val60.put(_key63, _val64);
+                      }
+                      iprot.readMapEnd();
+                    }
+                    struct.files.put(_key59, _val60);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setFilesIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // 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, loadFiles_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(TID_FIELD_DESC);
+        oprot.writeI64(struct.tid);
+        oprot.writeFieldEnd();
+        if (struct.dir != null) {
+          oprot.writeFieldBegin(DIR_FIELD_DESC);
+          oprot.writeString(struct.dir);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
+        oprot.writeBool(struct.setTime);
+        oprot.writeFieldEnd();
+        if (struct.files != null) {
+          oprot.writeFieldBegin(FILES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, struct.files.size()));
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,DataFileInfo>> _iter66 : struct.files.entrySet())
+            {
+              _iter66.getKey().write(oprot);
+              {
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter66.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, DataFileInfo> _iter67 : _iter66.getValue().entrySet())
+                {
+                  oprot.writeString(_iter67.getKey());
+                  _iter67.getValue().write(oprot);
+                }
+                oprot.writeMapEnd();
+              }
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class loadFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public loadFiles_argsTupleScheme getScheme() {
+        return new loadFiles_argsTupleScheme();
+      }
+    }
+
+    private static class loadFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<loadFiles_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, loadFiles_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.isSetDir()) {
+          optionals.set(3);
+        }
+        if (struct.isSetFiles()) {
+          optionals.set(4);
+        }
+        if (struct.isSetSetTime()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetTid()) {
+          oprot.writeI64(struct.tid);
+        }
+        if (struct.isSetDir()) {
+          oprot.writeString(struct.dir);
+        }
+        if (struct.isSetFiles()) {
+          {
+            oprot.writeI32(struct.files.size());
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,DataFileInfo>> _iter68 : struct.files.entrySet())
+            {
+              _iter68.getKey().write(oprot);
+              {
+                oprot.writeI32(_iter68.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, DataFileInfo> _iter69 : _iter68.getValue().entrySet())
+                {
+                  oprot.writeString(_iter69.getKey());
+                  _iter69.getValue().write(oprot);
+                }
+              }
+            }
+          }
+        }
+        if (struct.isSetSetTime()) {
+          oprot.writeBool(struct.setTime);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, loadFiles_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.tid = iprot.readI64();
+          struct.setTidIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.dir = iprot.readString();
+          struct.setDirIsSet(true);
+        }
+        if (incoming.get(4)) {
+          {
+            org.apache.thrift.protocol.TMap _map70 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP); 
+            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,DataFileInfo>>(2*_map70.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key71;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,DataFileInfo> _val72;
+            for (int _i73 = 0; _i73 < _map70.size; ++_i73)
+            {
+              _key71 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+              _key71.read(iprot);
+              {
+                org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
+                _val72 = new java.util.HashMap<java.lang.String,DataFileInfo>(2*_map74.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key75;
+                @org.apache.thrift.annotation.Nullable DataFileInfo _val76;
+                for (int _i77 = 0; _i77 < _map74.size; ++_i77)
+                {
+                  _key75 = iprot.readString();
+                  _val76 = new DataFileInfo();
+                  _val76.read(iprot);
+                  _val72.put(_key75, _val76);
+                }
+              }
+              struct.files.put(_key71, _val72);
+            }
+          }
+          struct.setFilesIsSet(true);
+        }
+        if (incoming.get(5)) {
+          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();
+    }
+  }
+
+  private static void unusedMethod() {}
+}
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveScan.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ActiveScan.java
similarity index 92%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveScan.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ActiveScan.java
index fddf9b2..248e588 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveScan.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ActiveScan.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.tabletscan.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class ActiveScan implements org.apache.thrift.TBase<ActiveScan, ActiveScan._Fields>, java.io.Serializable, Cloneable, Comparable<ActiveScan> {
@@ -1545,7 +1545,7 @@
             break;
           case 7: // TYPE
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.type = org.apache.accumulo.core.tabletserver.thrift.ScanType.findByValue(iprot.readI32());
+              struct.type = org.apache.accumulo.core.tabletscan.thrift.ScanType.findByValue(iprot.readI32());
               struct.setTypeIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -1553,7 +1553,7 @@
             break;
           case 8: // STATE
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.state = org.apache.accumulo.core.tabletserver.thrift.ScanState.findByValue(iprot.readI32());
+              struct.state = org.apache.accumulo.core.tabletscan.thrift.ScanState.findByValue(iprot.readI32());
               struct.setStateIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -1571,14 +1571,14 @@
           case 10: // COLUMNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
-                struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list8.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem9;
-                for (int _i10 = 0; _i10 < _list8.size; ++_i10)
+                org.apache.thrift.protocol.TList _list10 = iprot.readListBegin();
+                struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list10.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem11;
+                for (int _i12 = 0; _i12 < _list10.size; ++_i12)
                 {
-                  _elem9 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                  _elem9.read(iprot);
-                  struct.columns.add(_elem9);
+                  _elem11 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+                  _elem11.read(iprot);
+                  struct.columns.add(_elem11);
                 }
                 iprot.readListEnd();
               }
@@ -1590,14 +1590,14 @@
           case 11: // SSI_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list11 = iprot.readListBegin();
-                struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list11.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem12;
-                for (int _i13 = 0; _i13 < _list11.size; ++_i13)
+                org.apache.thrift.protocol.TList _list13 = iprot.readListBegin();
+                struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list13.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem14;
+                for (int _i15 = 0; _i15 < _list13.size; ++_i15)
                 {
-                  _elem12 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                  _elem12.read(iprot);
-                  struct.ssiList.add(_elem12);
+                  _elem14 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+                  _elem14.read(iprot);
+                  struct.ssiList.add(_elem14);
                 }
                 iprot.readListEnd();
               }
@@ -1609,27 +1609,27 @@
           case 12: // SSIO
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map14 = iprot.readMapBegin();
-                struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map14.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key15;
-                @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val16;
-                for (int _i17 = 0; _i17 < _map14.size; ++_i17)
+                org.apache.thrift.protocol.TMap _map16 = iprot.readMapBegin();
+                struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map16.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key17;
+                @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val18;
+                for (int _i19 = 0; _i19 < _map16.size; ++_i19)
                 {
-                  _key15 = iprot.readString();
+                  _key17 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map18 = iprot.readMapBegin();
-                    _val16 = 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)
+                    org.apache.thrift.protocol.TMap _map20 = iprot.readMapBegin();
+                    _val18 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map20.size);
+                    @org.apache.thrift.annotation.Nullable java.lang.String _key21;
+                    @org.apache.thrift.annotation.Nullable java.lang.String _val22;
+                    for (int _i23 = 0; _i23 < _map20.size; ++_i23)
                     {
-                      _key19 = iprot.readString();
-                      _val20 = iprot.readString();
-                      _val16.put(_key19, _val20);
+                      _key21 = iprot.readString();
+                      _val22 = iprot.readString();
+                      _val18.put(_key21, _val22);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.ssio.put(_key15, _val16);
+                  struct.ssio.put(_key17, _val18);
                 }
                 iprot.readMapEnd();
               }
@@ -1641,13 +1641,13 @@
           case 13: // AUTHORIZATIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list22 = iprot.readListBegin();
-                struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list22.size);
-                @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem23;
-                for (int _i24 = 0; _i24 < _list22.size; ++_i24)
+                org.apache.thrift.protocol.TList _list24 = iprot.readListBegin();
+                struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list24.size);
+                @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem25;
+                for (int _i26 = 0; _i26 < _list24.size; ++_i26)
                 {
-                  _elem23 = iprot.readBinary();
-                  struct.authorizations.add(_elem23);
+                  _elem25 = iprot.readBinary();
+                  struct.authorizations.add(_elem25);
                 }
                 iprot.readListEnd();
               }
@@ -1728,9 +1728,9 @@
         oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
-          for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter25 : struct.columns)
+          for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter27 : struct.columns)
           {
-            _iter25.write(oprot);
+            _iter27.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1740,9 +1740,9 @@
         oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter26 : struct.ssiList)
+          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter28 : struct.ssiList)
           {
-            _iter26.write(oprot);
+            _iter28.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1752,15 +1752,15 @@
         oprot.writeFieldBegin(SSIO_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter27 : struct.ssio.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter29 : struct.ssio.entrySet())
           {
-            oprot.writeString(_iter27.getKey());
+            oprot.writeString(_iter29.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter27.getValue().size()));
-              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter28 : _iter27.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter29.getValue().size()));
+              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter30 : _iter29.getValue().entrySet())
               {
-                oprot.writeString(_iter28.getKey());
-                oprot.writeString(_iter28.getValue());
+                oprot.writeString(_iter30.getKey());
+                oprot.writeString(_iter30.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1773,9 +1773,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 _iter29 : struct.authorizations)
+          for (java.nio.ByteBuffer _iter31 : struct.authorizations)
           {
-            oprot.writeBinary(_iter29);
+            oprot.writeBinary(_iter31);
           }
           oprot.writeListEnd();
         }
@@ -1880,33 +1880,33 @@
       if (struct.isSetColumns()) {
         {
           oprot.writeI32(struct.columns.size());
-          for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter30 : struct.columns)
+          for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter32 : struct.columns)
           {
-            _iter30.write(oprot);
+            _iter32.write(oprot);
           }
         }
       }
       if (struct.isSetSsiList()) {
         {
           oprot.writeI32(struct.ssiList.size());
-          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter31 : struct.ssiList)
+          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter33 : struct.ssiList)
           {
-            _iter31.write(oprot);
+            _iter33.write(oprot);
           }
         }
       }
       if (struct.isSetSsio()) {
         {
           oprot.writeI32(struct.ssio.size());
-          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter32 : struct.ssio.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter34 : struct.ssio.entrySet())
           {
-            oprot.writeString(_iter32.getKey());
+            oprot.writeString(_iter34.getKey());
             {
-              oprot.writeI32(_iter32.getValue().size());
-              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter33 : _iter32.getValue().entrySet())
+              oprot.writeI32(_iter34.getValue().size());
+              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter35 : _iter34.getValue().entrySet())
               {
-                oprot.writeString(_iter33.getKey());
-                oprot.writeString(_iter33.getValue());
+                oprot.writeString(_iter35.getKey());
+                oprot.writeString(_iter35.getValue());
               }
             }
           }
@@ -1915,9 +1915,9 @@
       if (struct.isSetAuthorizations()) {
         {
           oprot.writeI32(struct.authorizations.size());
-          for (java.nio.ByteBuffer _iter34 : struct.authorizations)
+          for (java.nio.ByteBuffer _iter36 : struct.authorizations)
           {
-            oprot.writeBinary(_iter34);
+            oprot.writeBinary(_iter36);
           }
         }
       }
@@ -1954,11 +1954,11 @@
         struct.setIdleTimeIsSet(true);
       }
       if (incoming.get(5)) {
-        struct.type = org.apache.accumulo.core.tabletserver.thrift.ScanType.findByValue(iprot.readI32());
+        struct.type = org.apache.accumulo.core.tabletscan.thrift.ScanType.findByValue(iprot.readI32());
         struct.setTypeIsSet(true);
       }
       if (incoming.get(6)) {
-        struct.state = org.apache.accumulo.core.tabletserver.thrift.ScanState.findByValue(iprot.readI32());
+        struct.state = org.apache.accumulo.core.tabletscan.thrift.ScanState.findByValue(iprot.readI32());
         struct.setStateIsSet(true);
       }
       if (incoming.get(7)) {
@@ -1968,67 +1968,67 @@
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TList _list35 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list35.size);
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem36;
-          for (int _i37 = 0; _i37 < _list35.size; ++_i37)
+          org.apache.thrift.protocol.TList _list37 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list37.size);
+          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem38;
+          for (int _i39 = 0; _i39 < _list37.size; ++_i39)
           {
-            _elem36 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-            _elem36.read(iprot);
-            struct.columns.add(_elem36);
+            _elem38 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+            _elem38.read(iprot);
+            struct.columns.add(_elem38);
           }
         }
         struct.setColumnsIsSet(true);
       }
       if (incoming.get(9)) {
         {
-          org.apache.thrift.protocol.TList _list38 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list38.size);
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem39;
-          for (int _i40 = 0; _i40 < _list38.size; ++_i40)
+          org.apache.thrift.protocol.TList _list40 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list40.size);
+          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem41;
+          for (int _i42 = 0; _i42 < _list40.size; ++_i42)
           {
-            _elem39 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-            _elem39.read(iprot);
-            struct.ssiList.add(_elem39);
+            _elem41 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+            _elem41.read(iprot);
+            struct.ssiList.add(_elem41);
           }
         }
         struct.setSsiListIsSet(true);
       }
       if (incoming.get(10)) {
         {
-          org.apache.thrift.protocol.TMap _map41 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
-          struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map41.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key42;
-          @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val43;
-          for (int _i44 = 0; _i44 < _map41.size; ++_i44)
+          org.apache.thrift.protocol.TMap _map43 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
+          struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map43.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key44;
+          @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val45;
+          for (int _i46 = 0; _i46 < _map43.size; ++_i46)
           {
-            _key42 = iprot.readString();
+            _key44 = iprot.readString();
             {
-              org.apache.thrift.protocol.TMap _map45 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-              _val43 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map45.size);
-              @org.apache.thrift.annotation.Nullable java.lang.String _key46;
-              @org.apache.thrift.annotation.Nullable java.lang.String _val47;
-              for (int _i48 = 0; _i48 < _map45.size; ++_i48)
+              org.apache.thrift.protocol.TMap _map47 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+              _val45 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map47.size);
+              @org.apache.thrift.annotation.Nullable java.lang.String _key48;
+              @org.apache.thrift.annotation.Nullable java.lang.String _val49;
+              for (int _i50 = 0; _i50 < _map47.size; ++_i50)
               {
-                _key46 = iprot.readString();
-                _val47 = iprot.readString();
-                _val43.put(_key46, _val47);
+                _key48 = iprot.readString();
+                _val49 = iprot.readString();
+                _val45.put(_key48, _val49);
               }
             }
-            struct.ssio.put(_key42, _val43);
+            struct.ssio.put(_key44, _val45);
           }
         }
         struct.setSsioIsSet(true);
       }
       if (incoming.get(11)) {
         {
-          org.apache.thrift.protocol.TList _list49 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-          struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list49.size);
-          @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem50;
-          for (int _i51 = 0; _i51 < _list49.size; ++_i51)
+          org.apache.thrift.protocol.TList _list51 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+          struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list51.size);
+          @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem52;
+          for (int _i53 = 0; _i53 < _list51.size; ++_i53)
           {
-            _elem50 = iprot.readBinary();
-            struct.authorizations.add(_elem50);
+            _elem52 = iprot.readBinary();
+            struct.authorizations.add(_elem52);
           }
         }
         struct.setAuthorizationsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanServerBusyException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanServerBusyException.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanServerBusyException.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanServerBusyException.java
index 894136d..ca3c6d1 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanServerBusyException.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanServerBusyException.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.tabletscan.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class ScanServerBusyException extends org.apache.thrift.TException implements org.apache.thrift.TBase<ScanServerBusyException, ScanServerBusyException._Fields>, java.io.Serializable, Cloneable, Comparable<ScanServerBusyException> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanState.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanState.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanState.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanState.java
index 5f20e90..7ace32a 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanState.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanState.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.tabletscan.thrift;
 
 
 public enum ScanState implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanType.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanType.java
similarity index 96%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanType.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanType.java
index 6c31229..168b5d6 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ScanType.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/ScanType.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.tabletscan.thrift;
 
 
 public enum ScanType implements org.apache.thrift.TEnum {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TSampleNotPresentException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TSampleNotPresentException.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TSampleNotPresentException.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TSampleNotPresentException.java
index 061ba51..3d0ea9b 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TSampleNotPresentException.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TSampleNotPresentException.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.tabletscan.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TSampleNotPresentException extends org.apache.thrift.TException implements org.apache.thrift.TBase<TSampleNotPresentException, TSampleNotPresentException._Fields>, java.io.Serializable, Cloneable, Comparable<TSampleNotPresentException> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TSamplerConfiguration.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TSamplerConfiguration.java
similarity index 93%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TSamplerConfiguration.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TSamplerConfiguration.java
index 607dac2..ff870bb 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TSamplerConfiguration.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TSamplerConfiguration.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.tabletscan.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TSamplerConfiguration implements org.apache.thrift.TBase<TSamplerConfiguration, TSamplerConfiguration._Fields>, java.io.Serializable, Cloneable, Comparable<TSamplerConfiguration> {
@@ -437,15 +437,15 @@
           case 2: // OPTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map106 = iprot.readMapBegin();
-                struct.options = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map106.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key107;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val108;
-                for (int _i109 = 0; _i109 < _map106.size; ++_i109)
+                org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin();
+                struct.options = 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)
                 {
-                  _key107 = iprot.readString();
-                  _val108 = iprot.readString();
-                  struct.options.put(_key107, _val108);
+                  _key1 = iprot.readString();
+                  _val2 = iprot.readString();
+                  struct.options.put(_key1, _val2);
                 }
                 iprot.readMapEnd();
               }
@@ -479,10 +479,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> _iter110 : struct.options.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter4 : struct.options.entrySet())
           {
-            oprot.writeString(_iter110.getKey());
-            oprot.writeString(_iter110.getValue());
+            oprot.writeString(_iter4.getKey());
+            oprot.writeString(_iter4.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -520,10 +520,10 @@
       if (struct.isSetOptions()) {
         {
           oprot.writeI32(struct.options.size());
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter111 : struct.options.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter5 : struct.options.entrySet())
           {
-            oprot.writeString(_iter111.getKey());
-            oprot.writeString(_iter111.getValue());
+            oprot.writeString(_iter5.getKey());
+            oprot.writeString(_iter5.getValue());
           }
         }
       }
@@ -539,15 +539,15 @@
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map112 = 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*_map112.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key113;
-          @org.apache.thrift.annotation.Nullable java.lang.String _val114;
-          for (int _i115 = 0; _i115 < _map112.size; ++_i115)
+          org.apache.thrift.protocol.TMap _map6 = 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*_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)
           {
-            _key113 = iprot.readString();
-            _val114 = iprot.readString();
-            struct.options.put(_key113, _val114);
+            _key7 = iprot.readString();
+            _val8 = iprot.readString();
+            struct.options.put(_key7, _val8);
           }
         }
         struct.setOptionsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletScanClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TabletScanClientService.java
similarity index 90%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletScanClientService.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TabletScanClientService.java
index 5fd64fd..c84c8d6 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletScanClientService.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TabletScanClientService.java
@@ -22,44 +22,44 @@
  * 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.tabletscan.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TabletScanClientService {
 
   public interface Iface {
 
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
 
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout) throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.dataImpl.thrift.ScanResult continueScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
 
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException;
+    public void closeScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException;
 
-    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
 
-    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout) throws NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
+    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException;
 
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, org.apache.thrift.TException;
+    public void closeMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException;
 
-    public java.util.List<ActiveScan> getActiveScans(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<ActiveScan> getActiveScans(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 startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException;
+    public void startScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException;
 
-    public void continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException;
+    public void continueScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException;
 
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void closeScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException;
+    public void startMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException;
 
-    public void continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException;
+    public void continueMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException;
 
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+    public void closeMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
 
-    public void getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException;
+    public void getActiveScans(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -86,13 +86,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.InitialScan startScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       send_startScan(tinfo, credentials, extent, range, columns, batchSize, ssiList, ssio, authorizations, waitForWrites, isolated, readaheadThreshold, samplerConfig, batchTimeOut, classLoaderContext, executionHints, busyTimeout);
       return recv_startScan();
     }
 
-    public void send_startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.thrift.TException
+    public void send_startScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.thrift.TException
     {
       startScan_args args = new startScan_args();
       args.setTinfo(tinfo);
@@ -115,7 +115,7 @@
       sendBase("startScan", args);
     }
 
-    public org.apache.accumulo.core.dataImpl.thrift.InitialScan recv_startScan() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.InitialScan recv_startScan() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       startScan_result result = new startScan_result();
       receiveBase(result, "startScan");
@@ -141,13 +141,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout) throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.ScanResult continueScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       send_continueScan(tinfo, scanID, busyTimeout);
       return recv_continueScan();
     }
 
-    public void send_continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.thrift.TException
+    public void send_continueScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.thrift.TException
     {
       continueScan_args args = new continueScan_args();
       args.setTinfo(tinfo);
@@ -156,7 +156,7 @@
       sendBase("continueScan", args);
     }
 
-    public org.apache.accumulo.core.dataImpl.thrift.ScanResult recv_continueScan() throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.ScanResult recv_continueScan() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       continueScan_result result = new continueScan_result();
       receiveBase(result, "continueScan");
@@ -182,12 +182,12 @@
     }
 
     @Override
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
+    public void closeScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
     {
       send_closeScan(tinfo, scanID);
     }
 
-    public void send_closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
+    public void send_closeScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
     {
       closeScan_args args = new closeScan_args();
       args.setTinfo(tinfo);
@@ -196,13 +196,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan startMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       send_startMultiScan(tinfo, credentials, batch, columns, ssiList, ssio, authorizations, waitForWrites, samplerConfig, batchTimeOut, classLoaderContext, executionHints, busyTimeout);
       return recv_startMultiScan();
     }
 
-    public void send_startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.thrift.TException
+    public void send_startMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout) throws org.apache.thrift.TException
     {
       startMultiScan_args args = new startMultiScan_args();
       args.setTinfo(tinfo);
@@ -241,13 +241,13 @@
     }
 
     @Override
-    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout) throws NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult continueMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       send_continueMultiScan(tinfo, scanID, busyTimeout);
       return recv_continueMultiScan();
     }
 
-    public void send_continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.thrift.TException
+    public void send_continueMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout) throws org.apache.thrift.TException
     {
       continueMultiScan_args args = new continueMultiScan_args();
       args.setTinfo(tinfo);
@@ -256,7 +256,7 @@
       sendBase("continueMultiScan", args);
     }
 
-    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult recv_continueMultiScan() throws NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
+    public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult recv_continueMultiScan() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException
     {
       continueMultiScan_result result = new continueMultiScan_result();
       receiveBase(result, "continueMultiScan");
@@ -276,13 +276,13 @@
     }
 
     @Override
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws NoSuchScanIDException, org.apache.thrift.TException
+    public void closeMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID) throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
     {
       send_closeMultiScan(tinfo, scanID);
       recv_closeMultiScan();
     }
 
-    public void send_closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
+    public void send_closeMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID) throws org.apache.thrift.TException
     {
       closeMultiScan_args args = new closeMultiScan_args();
       args.setTinfo(tinfo);
@@ -290,7 +290,7 @@
       sendBase("closeMultiScan", args);
     }
 
-    public void recv_closeMultiScan() throws NoSuchScanIDException, org.apache.thrift.TException
+    public void recv_closeMultiScan() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.thrift.TException
     {
       closeMultiScan_result result = new closeMultiScan_result();
       receiveBase(result, "closeMultiScan");
@@ -301,13 +301,13 @@
     }
 
     @Override
-    public java.util.List<ActiveScan> getActiveScans(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<ActiveScan> getActiveScans(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_getActiveScans(tinfo, credentials);
       return recv_getActiveScans();
     }
 
-    public void send_getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_getActiveScans(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       getActiveScans_args args = new getActiveScans_args();
       args.setTinfo(tinfo);
@@ -348,7 +348,7 @@
     }
 
     @Override
-    public void startScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException {
+    public void startScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       startScan_call method_call = new startScan_call(tinfo, credentials, extent, range, columns, batchSize, ssiList, ssio, authorizations, waitForWrites, isolated, readaheadThreshold, samplerConfig, batchTimeOut, classLoaderContext, executionHints, busyTimeout, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -356,7 +356,7 @@
     }
 
     public static class startScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.InitialScan> {
-      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.dataImpl.thrift.TKeyExtent extent;
       private org.apache.accumulo.core.dataImpl.thrift.TRange range;
@@ -373,7 +373,7 @@
       private java.lang.String classLoaderContext;
       private java.util.Map<java.lang.String,java.lang.String> executionHints;
       private long busyTimeout;
-      public startScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> 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 startScan_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, org.apache.accumulo.core.dataImpl.thrift.TRange range, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, int batchSize, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, boolean isolated, long readaheadThreshold, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialScan> 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;
@@ -420,7 +420,7 @@
       }
 
       @Override
-      public org.apache.accumulo.core.dataImpl.thrift.InitialScan getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException {
+      public org.apache.accumulo.core.dataImpl.thrift.InitialScan getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
@@ -431,7 +431,7 @@
     }
 
     @Override
-    public void continueScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException {
+    public void continueScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       continueScan_call method_call = new continueScan_call(tinfo, scanID, busyTimeout, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -439,10 +439,10 @@
     }
 
     public static class continueScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.ScanResult> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private long scanID;
       private long busyTimeout;
-      public continueScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> 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 continueScan_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.ScanResult> 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.scanID = scanID;
@@ -461,7 +461,7 @@
       }
 
       @Override
-      public org.apache.accumulo.core.dataImpl.thrift.ScanResult getResult() throws NoSuchScanIDException, NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException {
+      public org.apache.accumulo.core.dataImpl.thrift.ScanResult getResult() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException, TooManyFilesException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
@@ -472,7 +472,7 @@
     }
 
     @Override
-    public void closeScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void closeScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       closeScan_call method_call = new closeScan_call(tinfo, scanID, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -480,9 +480,9 @@
     }
 
     public static class closeScan_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 long scanID;
-      public closeScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, 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 closeScan_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, 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.scanID = scanID;
@@ -510,7 +510,7 @@
     }
 
     @Override
-    public void startMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException {
+    public void startMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       startMultiScan_call method_call = new startMultiScan_call(tinfo, credentials, batch, columns, ssiList, ssio, authorizations, waitForWrites, samplerConfig, batchTimeOut, classLoaderContext, executionHints, busyTimeout, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -518,7 +518,7 @@
     }
 
     public static class startMultiScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> {
-      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.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch;
       private java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns;
@@ -531,7 +531,7 @@
       private java.lang.String classLoaderContext;
       private java.util.Map<java.lang.String,java.lang.String> executionHints;
       private long busyTimeout;
-      public startMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> 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 startMultiScan_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns, java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList, java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio, java.util.List<java.nio.ByteBuffer> authorizations, boolean waitForWrites, TSamplerConfiguration samplerConfig, long batchTimeOut, java.lang.String classLoaderContext, java.util.Map<java.lang.String,java.lang.String> executionHints, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan> 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;
@@ -581,7 +581,7 @@
     }
 
     @Override
-    public void continueMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException {
+    public void continueMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       continueMultiScan_call method_call = new continueMultiScan_call(tinfo, scanID, busyTimeout, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -589,10 +589,10 @@
     }
 
     public static class continueMultiScan_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
       private long scanID;
       private long busyTimeout;
-      public continueMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> 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 continueMultiScan_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, long busyTimeout, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.MultiScanResult> 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.scanID = scanID;
@@ -611,7 +611,7 @@
       }
 
       @Override
-      public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult getResult() throws NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException {
+      public org.apache.accumulo.core.dataImpl.thrift.MultiScanResult getResult() throws org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException, TSampleNotPresentException, ScanServerBusyException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new java.lang.IllegalStateException("Method call not finished!");
         }
@@ -622,7 +622,7 @@
     }
 
     @Override
-    public void closeMultiScan(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+    public void closeMultiScan(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       closeMultiScan_call method_call = new closeMultiScan_call(tinfo, scanID, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -630,9 +630,9 @@
     }
 
     public static class closeMultiScan_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 long scanID;
-      public closeMultiScan_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long scanID, 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 closeMultiScan_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long scanID, 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.scanID = scanID;
@@ -649,7 +649,7 @@
       }
 
       @Override
-      public Void getResult() throws NoSuchScanIDException, org.apache.thrift.TException {
+      public Void 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!");
         }
@@ -661,7 +661,7 @@
     }
 
     @Override
-    public void getActiveScans(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException {
+    public void getActiveScans(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveScan>> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getActiveScans_call method_call = new getActiveScans_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
@@ -669,9 +669,9 @@
     }
 
     public static class getActiveScans_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<ActiveScan>> {
-      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 getActiveScans_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<ActiveScan>> 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 getActiveScans_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<ActiveScan>> 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;
@@ -748,7 +748,7 @@
           result.success = iface.startScan(args.tinfo, args.credentials, args.extent, args.range, args.columns, args.batchSize, args.ssiList, args.ssio, args.authorizations, args.waitForWrites, args.isolated, args.readaheadThreshold, args.samplerConfig, args.batchTimeOut, args.classLoaderContext, args.executionHints, args.busyTimeout);
         } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
-        } catch (NotServingTabletException nste) {
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
           result.nste = nste;
         } catch (TooManyFilesException tmfe) {
           result.tmfe = tmfe;
@@ -786,9 +786,9 @@
         continueScan_result result = new continueScan_result();
         try {
           result.success = iface.continueScan(args.tinfo, args.scanID, args.busyTimeout);
-        } catch (NoSuchScanIDException nssi) {
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
           result.nssi = nssi;
-        } catch (NotServingTabletException nste) {
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
           result.nste = nste;
         } catch (TooManyFilesException tmfe) {
           result.tmfe = tmfe;
@@ -889,7 +889,7 @@
         continueMultiScan_result result = new continueMultiScan_result();
         try {
           result.success = iface.continueMultiScan(args.tinfo, args.scanID, args.busyTimeout);
-        } catch (NoSuchScanIDException nssi) {
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
           result.nssi = nssi;
         } catch (TSampleNotPresentException tsnpe) {
           result.tsnpe = tsnpe;
@@ -925,7 +925,7 @@
         closeMultiScan_result result = new closeMultiScan_result();
         try {
           iface.closeMultiScan(args.tinfo, args.scanID);
-        } catch (NoSuchScanIDException nssi) {
+        } catch (org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
           result.nssi = nssi;
         }
         return result;
@@ -1024,8 +1024,8 @@
               result.sec = (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException) e;
               result.setSecIsSet(true);
               msg = result;
-            } else if (e instanceof NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
+            } 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 TooManyFilesException) {
@@ -1107,12 +1107,12 @@
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
             continueScan_result result = new continueScan_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
+            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 NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
+            } 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 TooManyFilesException) {
@@ -1313,8 +1313,8 @@
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
             continueMultiScan_result result = new continueMultiScan_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
+            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 TSampleNotPresentException) {
@@ -1391,8 +1391,8 @@
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TSerializable msg;
             closeMultiScan_result result = new closeMultiScan_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (NoSuchScanIDException) e;
+            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) {
@@ -1527,7 +1527,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startScan_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startScan_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.dataImpl.thrift.TKeyExtent extent; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange range; // required
@@ -1667,7 +1667,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.EXTENT, new org.apache.thrift.meta_data.FieldMetaData("extent", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -1717,7 +1717,7 @@
     }
 
     public startScan_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.dataImpl.thrift.TKeyExtent extent,
       org.apache.accumulo.core.dataImpl.thrift.TRange range,
@@ -1767,7 +1767,7 @@
     public startScan_args(startScan_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);
@@ -1862,11 +1862,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 startScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public startScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -2351,7 +2351,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;
 
@@ -3200,7 +3200,7 @@
           switch (schemeField.id) {
             case 11: // 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 { 
@@ -3237,14 +3237,14 @@
             case 4: // COLUMNS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list134 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list134.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem135;
-                  for (int _i136 = 0; _i136 < _list134.size; ++_i136)
+                  org.apache.thrift.protocol.TList _list54 = iprot.readListBegin();
+                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list54.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem55;
+                  for (int _i56 = 0; _i56 < _list54.size; ++_i56)
                   {
-                    _elem135 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                    _elem135.read(iprot);
-                    struct.columns.add(_elem135);
+                    _elem55 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+                    _elem55.read(iprot);
+                    struct.columns.add(_elem55);
                   }
                   iprot.readListEnd();
                 }
@@ -3264,14 +3264,14 @@
             case 6: // SSI_LIST
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list137 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list137.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem138;
-                  for (int _i139 = 0; _i139 < _list137.size; ++_i139)
+                  org.apache.thrift.protocol.TList _list57 = iprot.readListBegin();
+                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list57.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem58;
+                  for (int _i59 = 0; _i59 < _list57.size; ++_i59)
                   {
-                    _elem138 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                    _elem138.read(iprot);
-                    struct.ssiList.add(_elem138);
+                    _elem58 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+                    _elem58.read(iprot);
+                    struct.ssiList.add(_elem58);
                   }
                   iprot.readListEnd();
                 }
@@ -3283,27 +3283,27 @@
             case 7: // SSIO
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map140 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map140.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key141;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val142;
-                  for (int _i143 = 0; _i143 < _map140.size; ++_i143)
+                  org.apache.thrift.protocol.TMap _map60 = iprot.readMapBegin();
+                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map60.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key61;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val62;
+                  for (int _i63 = 0; _i63 < _map60.size; ++_i63)
                   {
-                    _key141 = iprot.readString();
+                    _key61 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TMap _map144 = iprot.readMapBegin();
-                      _val142 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map144.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key145;
-                      @org.apache.thrift.annotation.Nullable java.lang.String _val146;
-                      for (int _i147 = 0; _i147 < _map144.size; ++_i147)
+                      org.apache.thrift.protocol.TMap _map64 = iprot.readMapBegin();
+                      _val62 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map64.size);
+                      @org.apache.thrift.annotation.Nullable java.lang.String _key65;
+                      @org.apache.thrift.annotation.Nullable java.lang.String _val66;
+                      for (int _i67 = 0; _i67 < _map64.size; ++_i67)
                       {
-                        _key145 = iprot.readString();
-                        _val146 = iprot.readString();
-                        _val142.put(_key145, _val146);
+                        _key65 = iprot.readString();
+                        _val66 = iprot.readString();
+                        _val62.put(_key65, _val66);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.ssio.put(_key141, _val142);
+                    struct.ssio.put(_key61, _val62);
                   }
                   iprot.readMapEnd();
                 }
@@ -3315,13 +3315,13 @@
             case 8: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list148 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list148.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem149;
-                  for (int _i150 = 0; _i150 < _list148.size; ++_i150)
+                  org.apache.thrift.protocol.TList _list68 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list68.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem69;
+                  for (int _i70 = 0; _i70 < _list68.size; ++_i70)
                   {
-                    _elem149 = iprot.readBinary();
-                    struct.authorizations.add(_elem149);
+                    _elem69 = iprot.readBinary();
+                    struct.authorizations.add(_elem69);
                   }
                   iprot.readListEnd();
                 }
@@ -3382,15 +3382,15 @@
             case 16: // EXECUTION_HINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map151 = iprot.readMapBegin();
-                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map151.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key152;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val153;
-                  for (int _i154 = 0; _i154 < _map151.size; ++_i154)
+                  org.apache.thrift.protocol.TMap _map71 = iprot.readMapBegin();
+                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map71.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key72;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val73;
+                  for (int _i74 = 0; _i74 < _map71.size; ++_i74)
                   {
-                    _key152 = iprot.readString();
-                    _val153 = iprot.readString();
-                    struct.executionHints.put(_key152, _val153);
+                    _key72 = iprot.readString();
+                    _val73 = iprot.readString();
+                    struct.executionHints.put(_key72, _val73);
                   }
                   iprot.readMapEnd();
                 }
@@ -3442,9 +3442,9 @@
           oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter155 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter75 : struct.columns)
             {
-              _iter155.write(oprot);
+              _iter75.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -3457,9 +3457,9 @@
           oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter156 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter76 : struct.ssiList)
             {
-              _iter156.write(oprot);
+              _iter76.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -3469,15 +3469,15 @@
           oprot.writeFieldBegin(SSIO_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter157 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter77 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter157.getKey());
+              oprot.writeString(_iter77.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter157.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter158 : _iter157.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter77.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter78 : _iter77.getValue().entrySet())
                 {
-                  oprot.writeString(_iter158.getKey());
-                  oprot.writeString(_iter158.getValue());
+                  oprot.writeString(_iter78.getKey());
+                  oprot.writeString(_iter78.getValue());
                 }
                 oprot.writeMapEnd();
               }
@@ -3490,9 +3490,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 _iter159 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter79 : struct.authorizations)
             {
-              oprot.writeBinary(_iter159);
+              oprot.writeBinary(_iter79);
             }
             oprot.writeListEnd();
           }
@@ -3529,10 +3529,10 @@
           oprot.writeFieldBegin(EXECUTION_HINTS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.executionHints.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter160 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter80 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter160.getKey());
-              oprot.writeString(_iter160.getValue());
+              oprot.writeString(_iter80.getKey());
+              oprot.writeString(_iter80.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -3627,9 +3627,9 @@
         if (struct.isSetColumns()) {
           {
             oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter161 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter81 : struct.columns)
             {
-              _iter161.write(oprot);
+              _iter81.write(oprot);
             }
           }
         }
@@ -3639,24 +3639,24 @@
         if (struct.isSetSsiList()) {
           {
             oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter162 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter82 : struct.ssiList)
             {
-              _iter162.write(oprot);
+              _iter82.write(oprot);
             }
           }
         }
         if (struct.isSetSsio()) {
           {
             oprot.writeI32(struct.ssio.size());
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter163 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter83 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter163.getKey());
+              oprot.writeString(_iter83.getKey());
               {
-                oprot.writeI32(_iter163.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter164 : _iter163.getValue().entrySet())
+                oprot.writeI32(_iter83.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter84 : _iter83.getValue().entrySet())
                 {
-                  oprot.writeString(_iter164.getKey());
-                  oprot.writeString(_iter164.getValue());
+                  oprot.writeString(_iter84.getKey());
+                  oprot.writeString(_iter84.getValue());
                 }
               }
             }
@@ -3665,9 +3665,9 @@
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter165 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter85 : struct.authorizations)
             {
-              oprot.writeBinary(_iter165);
+              oprot.writeBinary(_iter85);
             }
           }
         }
@@ -3692,10 +3692,10 @@
         if (struct.isSetExecutionHints()) {
           {
             oprot.writeI32(struct.executionHints.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter166 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter86 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter166.getKey());
-              oprot.writeString(_iter166.getValue());
+              oprot.writeString(_iter86.getKey());
+              oprot.writeString(_iter86.getValue());
             }
           }
         }
@@ -3709,7 +3709,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(17);
         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);
         }
@@ -3730,14 +3730,14 @@
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list167 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list167.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem168;
-            for (int _i169 = 0; _i169 < _list167.size; ++_i169)
+            org.apache.thrift.protocol.TList _list87 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list87.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem88;
+            for (int _i89 = 0; _i89 < _list87.size; ++_i89)
             {
-              _elem168 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-              _elem168.read(iprot);
-              struct.columns.add(_elem168);
+              _elem88 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+              _elem88.read(iprot);
+              struct.columns.add(_elem88);
             }
           }
           struct.setColumnsIsSet(true);
@@ -3748,53 +3748,53 @@
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list170 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list170.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem171;
-            for (int _i172 = 0; _i172 < _list170.size; ++_i172)
+            org.apache.thrift.protocol.TList _list90 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list90.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem91;
+            for (int _i92 = 0; _i92 < _list90.size; ++_i92)
             {
-              _elem171 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-              _elem171.read(iprot);
-              struct.ssiList.add(_elem171);
+              _elem91 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+              _elem91.read(iprot);
+              struct.ssiList.add(_elem91);
             }
           }
           struct.setSsiListIsSet(true);
         }
         if (incoming.get(7)) {
           {
-            org.apache.thrift.protocol.TMap _map173 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map173.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key174;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val175;
-            for (int _i176 = 0; _i176 < _map173.size; ++_i176)
+            org.apache.thrift.protocol.TMap _map93 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
+            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map93.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key94;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val95;
+            for (int _i96 = 0; _i96 < _map93.size; ++_i96)
             {
-              _key174 = iprot.readString();
+              _key94 = iprot.readString();
               {
-                org.apache.thrift.protocol.TMap _map177 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-                _val175 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map177.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key178;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val179;
-                for (int _i180 = 0; _i180 < _map177.size; ++_i180)
+                org.apache.thrift.protocol.TMap _map97 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+                _val95 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map97.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key98;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val99;
+                for (int _i100 = 0; _i100 < _map97.size; ++_i100)
                 {
-                  _key178 = iprot.readString();
-                  _val179 = iprot.readString();
-                  _val175.put(_key178, _val179);
+                  _key98 = iprot.readString();
+                  _val99 = iprot.readString();
+                  _val95.put(_key98, _val99);
                 }
               }
-              struct.ssio.put(_key174, _val175);
+              struct.ssio.put(_key94, _val95);
             }
           }
           struct.setSsioIsSet(true);
         }
         if (incoming.get(8)) {
           {
-            org.apache.thrift.protocol.TList _list181 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list181.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem182;
-            for (int _i183 = 0; _i183 < _list181.size; ++_i183)
+            org.apache.thrift.protocol.TList _list101 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list101.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem102;
+            for (int _i103 = 0; _i103 < _list101.size; ++_i103)
             {
-              _elem182 = iprot.readBinary();
-              struct.authorizations.add(_elem182);
+              _elem102 = iprot.readBinary();
+              struct.authorizations.add(_elem102);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -3826,15 +3826,15 @@
         }
         if (incoming.get(15)) {
           {
-            org.apache.thrift.protocol.TMap _map184 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map184.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key185;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val186;
-            for (int _i187 = 0; _i187 < _map184.size; ++_i187)
+            org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.executionHints = 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)
             {
-              _key185 = iprot.readString();
-              _val186 = iprot.readString();
-              struct.executionHints.put(_key185, _val186);
+              _key105 = iprot.readString();
+              _val106 = iprot.readString();
+              struct.executionHints.put(_key105, _val106);
             }
           }
           struct.setExecutionHintsIsSet(true);
@@ -3867,7 +3867,7 @@
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.InitialScan success; // required
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
-    public @org.apache.thrift.annotation.Nullable NotServingTabletException nste; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste; // required
     public @org.apache.thrift.annotation.Nullable TooManyFilesException tmfe; // required
     public @org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe; // required
     public @org.apache.thrift.annotation.Nullable ScanServerBusyException ssbe; // required
@@ -3958,7 +3958,7 @@
       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, NotServingTabletException.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException.class)));
       tmpMap.put(_Fields.TMFE, new org.apache.thrift.meta_data.FieldMetaData("tmfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TooManyFilesException.class)));
       tmpMap.put(_Fields.TSNPE, new org.apache.thrift.meta_data.FieldMetaData("tsnpe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -3975,7 +3975,7 @@
     public startScan_result(
       org.apache.accumulo.core.dataImpl.thrift.InitialScan success,
       org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
-      NotServingTabletException nste,
+      org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste,
       TooManyFilesException tmfe,
       TSampleNotPresentException tsnpe,
       ScanServerBusyException ssbe)
@@ -4000,7 +4000,7 @@
         this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
       }
       if (other.isSetNste()) {
-        this.nste = new NotServingTabletException(other.nste);
+        this.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException(other.nste);
       }
       if (other.isSetTmfe()) {
         this.tmfe = new TooManyFilesException(other.tmfe);
@@ -4079,11 +4079,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public NotServingTabletException getNste() {
+    public org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException getNste() {
       return this.nste;
     }
 
-    public startScan_result setNste(@org.apache.thrift.annotation.Nullable NotServingTabletException nste) {
+    public startScan_result setNste(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
       this.nste = nste;
       return this;
     }
@@ -4201,7 +4201,7 @@
         if (value == null) {
           unsetNste();
         } else {
-          setNste((NotServingTabletException)value);
+          setNste((org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException)value);
         }
         break;
 
@@ -4589,7 +4589,7 @@
               break;
             case 2: // NSTE
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nste = new NotServingTabletException();
+                struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
                 struct.nste.read(iprot);
                 struct.setNsteIsSet(true);
               } else { 
@@ -4742,7 +4742,7 @@
           struct.setSecIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.nste = new NotServingTabletException();
+          struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
           struct.nste.read(iprot);
           struct.setNsteIsSet(true);
         }
@@ -4780,7 +4780,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new continueScan_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new continueScan_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 long scanID; // required
     public long busyTimeout; // required
 
@@ -4860,7 +4860,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.SCAN_ID, new org.apache.thrift.meta_data.FieldMetaData("scanID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "ScanID")));
       tmpMap.put(_Fields.BUSY_TIMEOUT, new org.apache.thrift.meta_data.FieldMetaData("busyTimeout", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -4873,7 +4873,7 @@
     }
 
     public continueScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       long scanID,
       long busyTimeout)
     {
@@ -4891,7 +4891,7 @@
     public continueScan_args(continueScan_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);
       }
       this.scanID = other.scanID;
       this.busyTimeout = other.busyTimeout;
@@ -4912,11 +4912,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 continueScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public continueScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -4989,7 +4989,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;
 
@@ -5234,7 +5234,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 { 
@@ -5329,7 +5329,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);
         }
@@ -5364,8 +5364,8 @@
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new continueScan_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.ScanResult success; // required
-    public @org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi; // required
-    public @org.apache.thrift.annotation.Nullable NotServingTabletException nste; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste; // required
     public @org.apache.thrift.annotation.Nullable TooManyFilesException tmfe; // required
     public @org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe; // required
     public @org.apache.thrift.annotation.Nullable ScanServerBusyException ssbe; // required
@@ -5454,9 +5454,9 @@
       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.ScanResult.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, NoSuchScanIDException.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException.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, NotServingTabletException.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException.class)));
       tmpMap.put(_Fields.TMFE, new org.apache.thrift.meta_data.FieldMetaData("tmfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TooManyFilesException.class)));
       tmpMap.put(_Fields.TSNPE, new org.apache.thrift.meta_data.FieldMetaData("tsnpe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -5472,8 +5472,8 @@
 
     public continueScan_result(
       org.apache.accumulo.core.dataImpl.thrift.ScanResult success,
-      NoSuchScanIDException nssi,
-      NotServingTabletException nste,
+      org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi,
+      org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste,
       TooManyFilesException tmfe,
       TSampleNotPresentException tsnpe,
       ScanServerBusyException ssbe)
@@ -5495,10 +5495,10 @@
         this.success = new org.apache.accumulo.core.dataImpl.thrift.ScanResult(other.success);
       }
       if (other.isSetNssi()) {
-        this.nssi = new NoSuchScanIDException(other.nssi);
+        this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
       }
       if (other.isSetNste()) {
-        this.nste = new NotServingTabletException(other.nste);
+        this.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException(other.nste);
       }
       if (other.isSetTmfe()) {
         this.tmfe = new TooManyFilesException(other.tmfe);
@@ -5552,11 +5552,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public NoSuchScanIDException getNssi() {
+    public org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
       return this.nssi;
     }
 
-    public continueScan_result setNssi(@org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi) {
+    public continueScan_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
       this.nssi = nssi;
       return this;
     }
@@ -5577,11 +5577,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public NotServingTabletException getNste() {
+    public org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException getNste() {
       return this.nste;
     }
 
-    public continueScan_result setNste(@org.apache.thrift.annotation.Nullable NotServingTabletException nste) {
+    public continueScan_result setNste(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException nste) {
       this.nste = nste;
       return this;
     }
@@ -5691,7 +5691,7 @@
         if (value == null) {
           unsetNssi();
         } else {
-          setNssi((NoSuchScanIDException)value);
+          setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
         }
         break;
 
@@ -5699,7 +5699,7 @@
         if (value == null) {
           unsetNste();
         } else {
-          setNste((NotServingTabletException)value);
+          setNste((org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException)value);
         }
         break;
 
@@ -6078,7 +6078,7 @@
               break;
             case 1: // NSSI
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nssi = new NoSuchScanIDException();
+                struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
                 struct.nssi.read(iprot);
                 struct.setNssiIsSet(true);
               } else { 
@@ -6087,7 +6087,7 @@
               break;
             case 2: // NSTE
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nste = new NotServingTabletException();
+                struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
                 struct.nste.read(iprot);
                 struct.setNsteIsSet(true);
               } else { 
@@ -6235,12 +6235,12 @@
           struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.nssi = new NoSuchScanIDException();
+          struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
           struct.nssi.read(iprot);
           struct.setNssiIsSet(true);
         }
         if (incoming.get(2)) {
-          struct.nste = new NotServingTabletException();
+          struct.nste = new org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException();
           struct.nste.read(iprot);
           struct.setNsteIsSet(true);
         }
@@ -6277,7 +6277,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeScan_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeScan_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 long scanID; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -6352,7 +6352,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.SCAN_ID, new org.apache.thrift.meta_data.FieldMetaData("scanID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "ScanID")));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -6363,7 +6363,7 @@
     }
 
     public closeScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       long scanID)
     {
       this();
@@ -6378,7 +6378,7 @@
     public closeScan_args(closeScan_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);
       }
       this.scanID = other.scanID;
     }
@@ -6396,11 +6396,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 closeScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public closeScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -6450,7 +6450,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;
 
@@ -6657,7 +6657,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 { 
@@ -6735,7 +6735,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);
         }
@@ -6772,7 +6772,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startMultiScan_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startMultiScan_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.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch; // required
     public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns; // required
@@ -6893,7 +6893,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.BATCH, new org.apache.thrift.meta_data.FieldMetaData("batch", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -6935,7 +6935,7 @@
     }
 
     public startMultiScan_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.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> batch,
       java.util.List<org.apache.accumulo.core.dataImpl.thrift.TColumn> columns,
@@ -6974,7 +6974,7 @@
     public startMultiScan_args(startMultiScan_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);
@@ -7071,11 +7071,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 startMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public startMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -7477,7 +7477,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;
 
@@ -8162,7 +8162,7 @@
           switch (schemeField.id) {
             case 8: // 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 { 
@@ -8181,27 +8181,27 @@
             case 2: // BATCH
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map188 = iprot.readMapBegin();
-                  struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map188.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key189;
-                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val190;
-                  for (int _i191 = 0; _i191 < _map188.size; ++_i191)
+                  org.apache.thrift.protocol.TMap _map108 = iprot.readMapBegin();
+                  struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map108.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key109;
+                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val110;
+                  for (int _i111 = 0; _i111 < _map108.size; ++_i111)
                   {
-                    _key189 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key189.read(iprot);
+                    _key109 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+                    _key109.read(iprot);
                     {
-                      org.apache.thrift.protocol.TList _list192 = iprot.readListBegin();
-                      _val190 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list192.size);
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem193;
-                      for (int _i194 = 0; _i194 < _list192.size; ++_i194)
+                      org.apache.thrift.protocol.TList _list112 = iprot.readListBegin();
+                      _val110 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list112.size);
+                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem113;
+                      for (int _i114 = 0; _i114 < _list112.size; ++_i114)
                       {
-                        _elem193 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-                        _elem193.read(iprot);
-                        _val190.add(_elem193);
+                        _elem113 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
+                        _elem113.read(iprot);
+                        _val110.add(_elem113);
                       }
                       iprot.readListEnd();
                     }
-                    struct.batch.put(_key189, _val190);
+                    struct.batch.put(_key109, _val110);
                   }
                   iprot.readMapEnd();
                 }
@@ -8213,14 +8213,14 @@
             case 3: // COLUMNS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list195 = iprot.readListBegin();
-                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list195.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem196;
-                  for (int _i197 = 0; _i197 < _list195.size; ++_i197)
+                  org.apache.thrift.protocol.TList _list115 = iprot.readListBegin();
+                  struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list115.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem116;
+                  for (int _i117 = 0; _i117 < _list115.size; ++_i117)
                   {
-                    _elem196 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-                    _elem196.read(iprot);
-                    struct.columns.add(_elem196);
+                    _elem116 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+                    _elem116.read(iprot);
+                    struct.columns.add(_elem116);
                   }
                   iprot.readListEnd();
                 }
@@ -8232,14 +8232,14 @@
             case 4: // SSI_LIST
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list198 = iprot.readListBegin();
-                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list198.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem199;
-                  for (int _i200 = 0; _i200 < _list198.size; ++_i200)
+                  org.apache.thrift.protocol.TList _list118 = iprot.readListBegin();
+                  struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list118.size);
+                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem119;
+                  for (int _i120 = 0; _i120 < _list118.size; ++_i120)
                   {
-                    _elem199 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                    _elem199.read(iprot);
-                    struct.ssiList.add(_elem199);
+                    _elem119 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+                    _elem119.read(iprot);
+                    struct.ssiList.add(_elem119);
                   }
                   iprot.readListEnd();
                 }
@@ -8251,27 +8251,27 @@
             case 5: // SSIO
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map201 = iprot.readMapBegin();
-                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map201.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key202;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val203;
-                  for (int _i204 = 0; _i204 < _map201.size; ++_i204)
+                  org.apache.thrift.protocol.TMap _map121 = iprot.readMapBegin();
+                  struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map121.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key122;
+                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val123;
+                  for (int _i124 = 0; _i124 < _map121.size; ++_i124)
                   {
-                    _key202 = iprot.readString();
+                    _key122 = iprot.readString();
                     {
-                      org.apache.thrift.protocol.TMap _map205 = iprot.readMapBegin();
-                      _val203 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map205.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key206;
-                      @org.apache.thrift.annotation.Nullable java.lang.String _val207;
-                      for (int _i208 = 0; _i208 < _map205.size; ++_i208)
+                      org.apache.thrift.protocol.TMap _map125 = iprot.readMapBegin();
+                      _val123 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map125.size);
+                      @org.apache.thrift.annotation.Nullable java.lang.String _key126;
+                      @org.apache.thrift.annotation.Nullable java.lang.String _val127;
+                      for (int _i128 = 0; _i128 < _map125.size; ++_i128)
                       {
-                        _key206 = iprot.readString();
-                        _val207 = iprot.readString();
-                        _val203.put(_key206, _val207);
+                        _key126 = iprot.readString();
+                        _val127 = iprot.readString();
+                        _val123.put(_key126, _val127);
                       }
                       iprot.readMapEnd();
                     }
-                    struct.ssio.put(_key202, _val203);
+                    struct.ssio.put(_key122, _val123);
                   }
                   iprot.readMapEnd();
                 }
@@ -8283,13 +8283,13 @@
             case 6: // AUTHORIZATIONS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list209 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list209.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem210;
-                  for (int _i211 = 0; _i211 < _list209.size; ++_i211)
+                  org.apache.thrift.protocol.TList _list129 = iprot.readListBegin();
+                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list129.size);
+                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem130;
+                  for (int _i131 = 0; _i131 < _list129.size; ++_i131)
                   {
-                    _elem210 = iprot.readBinary();
-                    struct.authorizations.add(_elem210);
+                    _elem130 = iprot.readBinary();
+                    struct.authorizations.add(_elem130);
                   }
                   iprot.readListEnd();
                 }
@@ -8334,15 +8334,15 @@
             case 12: // EXECUTION_HINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map212 = iprot.readMapBegin();
-                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map212.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key213;
-                  @org.apache.thrift.annotation.Nullable java.lang.String _val214;
-                  for (int _i215 = 0; _i215 < _map212.size; ++_i215)
+                  org.apache.thrift.protocol.TMap _map132 = iprot.readMapBegin();
+                  struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map132.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key133;
+                  @org.apache.thrift.annotation.Nullable java.lang.String _val134;
+                  for (int _i135 = 0; _i135 < _map132.size; ++_i135)
                   {
-                    _key213 = iprot.readString();
-                    _val214 = iprot.readString();
-                    struct.executionHints.put(_key213, _val214);
+                    _key133 = iprot.readString();
+                    _val134 = iprot.readString();
+                    struct.executionHints.put(_key133, _val134);
                   }
                   iprot.readMapEnd();
                 }
@@ -8384,14 +8384,14 @@
           oprot.writeFieldBegin(BATCH_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST, struct.batch.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter216 : struct.batch.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter136 : struct.batch.entrySet())
             {
-              _iter216.getKey().write(oprot);
+              _iter136.getKey().write(oprot);
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter216.getValue().size()));
-                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter217 : _iter216.getValue())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter136.getValue().size()));
+                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter137 : _iter136.getValue())
                 {
-                  _iter217.write(oprot);
+                  _iter137.write(oprot);
                 }
                 oprot.writeListEnd();
               }
@@ -8404,9 +8404,9 @@
           oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter218 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter138 : struct.columns)
             {
-              _iter218.write(oprot);
+              _iter138.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -8416,9 +8416,9 @@
           oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter219 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter139 : struct.ssiList)
             {
-              _iter219.write(oprot);
+              _iter139.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -8428,15 +8428,15 @@
           oprot.writeFieldBegin(SSIO_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter220 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter140 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter220.getKey());
+              oprot.writeString(_iter140.getKey());
               {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter220.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter221 : _iter220.getValue().entrySet())
+                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter140.getValue().size()));
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter141 : _iter140.getValue().entrySet())
                 {
-                  oprot.writeString(_iter221.getKey());
-                  oprot.writeString(_iter221.getValue());
+                  oprot.writeString(_iter141.getKey());
+                  oprot.writeString(_iter141.getValue());
                 }
                 oprot.writeMapEnd();
               }
@@ -8449,9 +8449,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 _iter222 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter142 : struct.authorizations)
             {
-              oprot.writeBinary(_iter222);
+              oprot.writeBinary(_iter142);
             }
             oprot.writeListEnd();
           }
@@ -8482,10 +8482,10 @@
           oprot.writeFieldBegin(EXECUTION_HINTS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.executionHints.size()));
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter223 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter143 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter223.getKey());
-              oprot.writeString(_iter223.getValue());
+              oprot.writeString(_iter143.getKey());
+              oprot.writeString(_iter143.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -8562,14 +8562,14 @@
         if (struct.isSetBatch()) {
           {
             oprot.writeI32(struct.batch.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter224 : struct.batch.entrySet())
+            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>> _iter144 : struct.batch.entrySet())
             {
-              _iter224.getKey().write(oprot);
+              _iter144.getKey().write(oprot);
               {
-                oprot.writeI32(_iter224.getValue().size());
-                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter225 : _iter224.getValue())
+                oprot.writeI32(_iter144.getValue().size());
+                for (org.apache.accumulo.core.dataImpl.thrift.TRange _iter145 : _iter144.getValue())
                 {
-                  _iter225.write(oprot);
+                  _iter145.write(oprot);
                 }
               }
             }
@@ -8578,33 +8578,33 @@
         if (struct.isSetColumns()) {
           {
             oprot.writeI32(struct.columns.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter226 : struct.columns)
+            for (org.apache.accumulo.core.dataImpl.thrift.TColumn _iter146 : struct.columns)
             {
-              _iter226.write(oprot);
+              _iter146.write(oprot);
             }
           }
         }
         if (struct.isSetSsiList()) {
           {
             oprot.writeI32(struct.ssiList.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter227 : struct.ssiList)
+            for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter147 : struct.ssiList)
             {
-              _iter227.write(oprot);
+              _iter147.write(oprot);
             }
           }
         }
         if (struct.isSetSsio()) {
           {
             oprot.writeI32(struct.ssio.size());
-            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter228 : struct.ssio.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter148 : struct.ssio.entrySet())
             {
-              oprot.writeString(_iter228.getKey());
+              oprot.writeString(_iter148.getKey());
               {
-                oprot.writeI32(_iter228.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter229 : _iter228.getValue().entrySet())
+                oprot.writeI32(_iter148.getValue().size());
+                for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter149 : _iter148.getValue().entrySet())
                 {
-                  oprot.writeString(_iter229.getKey());
-                  oprot.writeString(_iter229.getValue());
+                  oprot.writeString(_iter149.getKey());
+                  oprot.writeString(_iter149.getValue());
                 }
               }
             }
@@ -8613,9 +8613,9 @@
         if (struct.isSetAuthorizations()) {
           {
             oprot.writeI32(struct.authorizations.size());
-            for (java.nio.ByteBuffer _iter230 : struct.authorizations)
+            for (java.nio.ByteBuffer _iter150 : struct.authorizations)
             {
-              oprot.writeBinary(_iter230);
+              oprot.writeBinary(_iter150);
             }
           }
         }
@@ -8634,10 +8634,10 @@
         if (struct.isSetExecutionHints()) {
           {
             oprot.writeI32(struct.executionHints.size());
-            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter231 : struct.executionHints.entrySet())
+            for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter151 : struct.executionHints.entrySet())
             {
-              oprot.writeString(_iter231.getKey());
-              oprot.writeString(_iter231.getValue());
+              oprot.writeString(_iter151.getKey());
+              oprot.writeString(_iter151.getValue());
             }
           }
         }
@@ -8651,7 +8651,7 @@
         org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
         java.util.BitSet incoming = iprot.readBitSet(13);
         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);
         }
@@ -8662,93 +8662,93 @@
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TMap _map232 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST); 
-            struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map232.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key233;
-            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val234;
-            for (int _i235 = 0; _i235 < _map232.size; ++_i235)
+            org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.LIST); 
+            struct.batch = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange>>(2*_map152.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key153;
+            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRange> _val154;
+            for (int _i155 = 0; _i155 < _map152.size; ++_i155)
             {
-              _key233 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key233.read(iprot);
+              _key153 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
+              _key153.read(iprot);
               {
-                org.apache.thrift.protocol.TList _list236 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-                _val234 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list236.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem237;
-                for (int _i238 = 0; _i238 < _list236.size; ++_i238)
+                org.apache.thrift.protocol.TList _list156 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+                _val154 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRange>(_list156.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRange _elem157;
+                for (int _i158 = 0; _i158 < _list156.size; ++_i158)
                 {
-                  _elem237 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
-                  _elem237.read(iprot);
-                  _val234.add(_elem237);
+                  _elem157 = new org.apache.accumulo.core.dataImpl.thrift.TRange();
+                  _elem157.read(iprot);
+                  _val154.add(_elem157);
                 }
               }
-              struct.batch.put(_key233, _val234);
+              struct.batch.put(_key153, _val154);
             }
           }
           struct.setBatchIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list239 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list239.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem240;
-            for (int _i241 = 0; _i241 < _list239.size; ++_i241)
+            org.apache.thrift.protocol.TList _list159 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.columns = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TColumn>(_list159.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TColumn _elem160;
+            for (int _i161 = 0; _i161 < _list159.size; ++_i161)
             {
-              _elem240 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
-              _elem240.read(iprot);
-              struct.columns.add(_elem240);
+              _elem160 = new org.apache.accumulo.core.dataImpl.thrift.TColumn();
+              _elem160.read(iprot);
+              struct.columns.add(_elem160);
             }
           }
           struct.setColumnsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list242 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list242.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem243;
-            for (int _i244 = 0; _i244 < _list242.size; ++_i244)
+            org.apache.thrift.protocol.TList _list162 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list162.size);
+            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem163;
+            for (int _i164 = 0; _i164 < _list162.size; ++_i164)
             {
-              _elem243 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-              _elem243.read(iprot);
-              struct.ssiList.add(_elem243);
+              _elem163 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+              _elem163.read(iprot);
+              struct.ssiList.add(_elem163);
             }
           }
           struct.setSsiListIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TMap _map245 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
-            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map245.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key246;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val247;
-            for (int _i248 = 0; _i248 < _map245.size; ++_i248)
+            org.apache.thrift.protocol.TMap _map165 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
+            struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map165.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key166;
+            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val167;
+            for (int _i168 = 0; _i168 < _map165.size; ++_i168)
             {
-              _key246 = iprot.readString();
+              _key166 = iprot.readString();
               {
-                org.apache.thrift.protocol.TMap _map249 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-                _val247 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map249.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key250;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val251;
-                for (int _i252 = 0; _i252 < _map249.size; ++_i252)
+                org.apache.thrift.protocol.TMap _map169 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+                _val167 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map169.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key170;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val171;
+                for (int _i172 = 0; _i172 < _map169.size; ++_i172)
                 {
-                  _key250 = iprot.readString();
-                  _val251 = iprot.readString();
-                  _val247.put(_key250, _val251);
+                  _key170 = iprot.readString();
+                  _val171 = iprot.readString();
+                  _val167.put(_key170, _val171);
                 }
               }
-              struct.ssio.put(_key246, _val247);
+              struct.ssio.put(_key166, _val167);
             }
           }
           struct.setSsioIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list253 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list253.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem254;
-            for (int _i255 = 0; _i255 < _list253.size; ++_i255)
+            org.apache.thrift.protocol.TList _list173 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list173.size);
+            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem174;
+            for (int _i175 = 0; _i175 < _list173.size; ++_i175)
             {
-              _elem254 = iprot.readBinary();
-              struct.authorizations.add(_elem254);
+              _elem174 = iprot.readBinary();
+              struct.authorizations.add(_elem174);
             }
           }
           struct.setAuthorizationsIsSet(true);
@@ -8772,15 +8772,15 @@
         }
         if (incoming.get(11)) {
           {
-            org.apache.thrift.protocol.TMap _map256 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map256.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key257;
-            @org.apache.thrift.annotation.Nullable java.lang.String _val258;
-            for (int _i259 = 0; _i259 < _map256.size; ++_i259)
+            org.apache.thrift.protocol.TMap _map176 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+            struct.executionHints = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map176.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key177;
+            @org.apache.thrift.annotation.Nullable java.lang.String _val178;
+            for (int _i179 = 0; _i179 < _map176.size; ++_i179)
             {
-              _key257 = iprot.readString();
-              _val258 = iprot.readString();
-              struct.executionHints.put(_key257, _val258);
+              _key177 = iprot.readString();
+              _val178 = iprot.readString();
+              struct.executionHints.put(_key177, _val178);
             }
           }
           struct.setExecutionHintsIsSet(true);
@@ -9512,7 +9512,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new continueMultiScan_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new continueMultiScan_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 long scanID; // required
     public long busyTimeout; // required
 
@@ -9592,7 +9592,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.SCAN_ID, new org.apache.thrift.meta_data.FieldMetaData("scanID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "ScanID")));
       tmpMap.put(_Fields.BUSY_TIMEOUT, new org.apache.thrift.meta_data.FieldMetaData("busyTimeout", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -9605,7 +9605,7 @@
     }
 
     public continueMultiScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       long scanID,
       long busyTimeout)
     {
@@ -9623,7 +9623,7 @@
     public continueMultiScan_args(continueMultiScan_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);
       }
       this.scanID = other.scanID;
       this.busyTimeout = other.busyTimeout;
@@ -9644,11 +9644,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 continueMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public continueMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -9721,7 +9721,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;
 
@@ -9966,7 +9966,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 { 
@@ -10061,7 +10061,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);
         }
@@ -10094,7 +10094,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new continueMultiScan_resultTupleSchemeFactory();
 
     public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MultiScanResult success; // required
-    public @org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi; // required
     public @org.apache.thrift.annotation.Nullable TSampleNotPresentException tsnpe; // required
     public @org.apache.thrift.annotation.Nullable ScanServerBusyException ssbe; // required
 
@@ -10176,7 +10176,7 @@
       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.MultiScanResult.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, NoSuchScanIDException.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException.class)));
       tmpMap.put(_Fields.TSNPE, new org.apache.thrift.meta_data.FieldMetaData("tsnpe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSampleNotPresentException.class)));
       tmpMap.put(_Fields.SSBE, new org.apache.thrift.meta_data.FieldMetaData("ssbe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -10190,7 +10190,7 @@
 
     public continueMultiScan_result(
       org.apache.accumulo.core.dataImpl.thrift.MultiScanResult success,
-      NoSuchScanIDException nssi,
+      org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi,
       TSampleNotPresentException tsnpe,
       ScanServerBusyException ssbe)
     {
@@ -10209,7 +10209,7 @@
         this.success = new org.apache.accumulo.core.dataImpl.thrift.MultiScanResult(other.success);
       }
       if (other.isSetNssi()) {
-        this.nssi = new NoSuchScanIDException(other.nssi);
+        this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
       }
       if (other.isSetTsnpe()) {
         this.tsnpe = new TSampleNotPresentException(other.tsnpe);
@@ -10258,11 +10258,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public NoSuchScanIDException getNssi() {
+    public org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
       return this.nssi;
     }
 
-    public continueMultiScan_result setNssi(@org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi) {
+    public continueMultiScan_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
       this.nssi = nssi;
       return this;
     }
@@ -10347,7 +10347,7 @@
         if (value == null) {
           unsetNssi();
         } else {
-          setNssi((NoSuchScanIDException)value);
+          setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
         }
         break;
 
@@ -10646,7 +10646,7 @@
               break;
             case 1: // NSSI
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nssi = new NoSuchScanIDException();
+                struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
                 struct.nssi.read(iprot);
                 struct.setNssiIsSet(true);
               } else { 
@@ -10763,7 +10763,7 @@
           struct.setSuccessIsSet(true);
         }
         if (incoming.get(1)) {
-          struct.nssi = new NoSuchScanIDException();
+          struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
           struct.nssi.read(iprot);
           struct.setNssiIsSet(true);
         }
@@ -10795,7 +10795,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeMultiScan_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeMultiScan_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 long scanID; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -10870,7 +10870,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.SCAN_ID, new org.apache.thrift.meta_data.FieldMetaData("scanID", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64          , "ScanID")));
       metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
@@ -10881,7 +10881,7 @@
     }
 
     public closeMultiScan_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
       long scanID)
     {
       this();
@@ -10896,7 +10896,7 @@
     public closeMultiScan_args(closeMultiScan_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);
       }
       this.scanID = other.scanID;
     }
@@ -10914,11 +10914,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 closeMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public closeMultiScan_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -10968,7 +10968,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;
 
@@ -11175,7 +11175,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 { 
@@ -11253,7 +11253,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);
         }
@@ -11278,7 +11278,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new closeMultiScan_resultStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new closeMultiScan_resultTupleSchemeFactory();
 
-    public @org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi; // 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 {
@@ -11347,7 +11347,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.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, NoSuchScanIDException.class)));
+          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(closeMultiScan_result.class, metaDataMap);
     }
@@ -11356,7 +11356,7 @@
     }
 
     public closeMultiScan_result(
-      NoSuchScanIDException nssi)
+      org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi)
     {
       this();
       this.nssi = nssi;
@@ -11367,7 +11367,7 @@
      */
     public closeMultiScan_result(closeMultiScan_result other) {
       if (other.isSetNssi()) {
-        this.nssi = new NoSuchScanIDException(other.nssi);
+        this.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException(other.nssi);
       }
     }
 
@@ -11382,11 +11382,11 @@
     }
 
     @org.apache.thrift.annotation.Nullable
-    public NoSuchScanIDException getNssi() {
+    public org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException getNssi() {
       return this.nssi;
     }
 
-    public closeMultiScan_result setNssi(@org.apache.thrift.annotation.Nullable NoSuchScanIDException nssi) {
+    public closeMultiScan_result setNssi(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException nssi) {
       this.nssi = nssi;
       return this;
     }
@@ -11413,7 +11413,7 @@
         if (value == null) {
           unsetNssi();
         } else {
-          setNssi((NoSuchScanIDException)value);
+          setNssi((org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException)value);
         }
         break;
 
@@ -11576,7 +11576,7 @@
           switch (schemeField.id) {
             case 1: // NSSI
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.nssi = new NoSuchScanIDException();
+                struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
                 struct.nssi.read(iprot);
                 struct.setNssiIsSet(true);
               } else { 
@@ -11637,7 +11637,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.nssi = new NoSuchScanIDException();
+          struct.nssi = new org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException();
           struct.nssi.read(iprot);
           struct.setNssiIsSet(true);
         }
@@ -11659,7 +11659,7 @@
     private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveScans_argsStandardSchemeFactory();
     private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveScans_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. */
@@ -11732,7 +11732,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);
@@ -11743,7 +11743,7 @@
     }
 
     public getActiveScans_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();
@@ -11756,7 +11756,7 @@
      */
     public getActiveScans_args(getActiveScans_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);
@@ -11775,11 +11775,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 getActiveScans_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+    public getActiveScans_args setTinfo(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -11831,7 +11831,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;
 
@@ -12045,7 +12045,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 { 
@@ -12126,7 +12126,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);
         }
@@ -12554,14 +12554,14 @@
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list260 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ActiveScan>(_list260.size);
-                  @org.apache.thrift.annotation.Nullable ActiveScan _elem261;
-                  for (int _i262 = 0; _i262 < _list260.size; ++_i262)
+                  org.apache.thrift.protocol.TList _list180 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<ActiveScan>(_list180.size);
+                  @org.apache.thrift.annotation.Nullable ActiveScan _elem181;
+                  for (int _i182 = 0; _i182 < _list180.size; ++_i182)
                   {
-                    _elem261 = new ActiveScan();
-                    _elem261.read(iprot);
-                    struct.success.add(_elem261);
+                    _elem181 = new ActiveScan();
+                    _elem181.read(iprot);
+                    struct.success.add(_elem181);
                   }
                   iprot.readListEnd();
                 }
@@ -12599,9 +12599,9 @@
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (ActiveScan _iter263 : struct.success)
+            for (ActiveScan _iter183 : struct.success)
             {
-              _iter263.write(oprot);
+              _iter183.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -12641,9 +12641,9 @@
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (ActiveScan _iter264 : struct.success)
+            for (ActiveScan _iter184 : struct.success)
             {
-              _iter264.write(oprot);
+              _iter184.write(oprot);
             }
           }
         }
@@ -12658,14 +12658,14 @@
         java.util.BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list265 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<ActiveScan>(_list265.size);
-            @org.apache.thrift.annotation.Nullable ActiveScan _elem266;
-            for (int _i267 = 0; _i267 < _list265.size; ++_i267)
+            org.apache.thrift.protocol.TList _list185 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<ActiveScan>(_list185.size);
+            @org.apache.thrift.annotation.Nullable ActiveScan _elem186;
+            for (int _i187 = 0; _i187 < _list185.size; ++_i187)
             {
-              _elem266 = new ActiveScan();
-              _elem266.read(iprot);
-              struct.success.add(_elem266);
+              _elem186 = new ActiveScan();
+              _elem186.read(iprot);
+              struct.success.add(_elem186);
             }
           }
           struct.setSuccessIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TooManyFilesException.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TooManyFilesException.java
similarity index 99%
rename from core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TooManyFilesException.java
rename to core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TooManyFilesException.java
index f4e5084..b66641c 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TooManyFilesException.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletscan/thrift/TooManyFilesException.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.tabletscan.thrift;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
 public class TooManyFilesException extends org.apache.thrift.TException implements org.apache.thrift.TBase<TooManyFilesException, TooManyFilesException._Fields>, java.io.Serializable, Cloneable, Comparable<TooManyFilesException> {
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java
index e44aec2..3c48fe1 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/ActiveCompaction.java
@@ -39,6 +39,7 @@
   private static final org.apache.thrift.protocol.TField ENTRIES_WRITTEN_FIELD_DESC = new org.apache.thrift.protocol.TField("entriesWritten", org.apache.thrift.protocol.TType.I64, (short)9);
   private static final org.apache.thrift.protocol.TField SSI_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("ssiList", org.apache.thrift.protocol.TType.LIST, (short)10);
   private static final org.apache.thrift.protocol.TField SSIO_FIELD_DESC = new org.apache.thrift.protocol.TField("ssio", org.apache.thrift.protocol.TType.MAP, (short)11);
+  private static final org.apache.thrift.protocol.TField TIMES_PAUSED_FIELD_DESC = new org.apache.thrift.protocol.TField("timesPaused", org.apache.thrift.protocol.TType.I64, (short)12);
 
   private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new ActiveCompactionStandardSchemeFactory();
   private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new ActiveCompactionTupleSchemeFactory();
@@ -62,6 +63,7 @@
   public long entriesWritten; // required
   public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList; // required
   public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio; // required
+  public long timesPaused; // 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 {
@@ -83,7 +85,8 @@
     ENTRIES_READ((short)8, "entriesRead"),
     ENTRIES_WRITTEN((short)9, "entriesWritten"),
     SSI_LIST((short)10, "ssiList"),
-    SSIO((short)11, "ssio");
+    SSIO((short)11, "ssio"),
+    TIMES_PAUSED((short)12, "timesPaused");
 
     private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
 
@@ -121,6 +124,8 @@
           return SSI_LIST;
         case 11: // SSIO
           return SSIO;
+        case 12: // TIMES_PAUSED
+          return TIMES_PAUSED;
         default:
           return null;
       }
@@ -167,6 +172,7 @@
   private static final int __AGE_ISSET_ID = 0;
   private static final int __ENTRIESREAD_ISSET_ID = 1;
   private static final int __ENTRIESWRITTEN_ISSET_ID = 2;
+  private static final int __TIMESPAUSED_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
   public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
@@ -199,6 +205,8 @@
             new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))));
+    tmpMap.put(_Fields.TIMES_PAUSED, new org.apache.thrift.meta_data.FieldMetaData("timesPaused", 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(ActiveCompaction.class, metaDataMap);
   }
@@ -217,7 +225,8 @@
     long entriesRead,
     long entriesWritten,
     java.util.List<org.apache.accumulo.core.dataImpl.thrift.IterInfo> ssiList,
-    java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio)
+    java.util.Map<java.lang.String,java.util.Map<java.lang.String,java.lang.String>> ssio,
+    long timesPaused)
   {
     this();
     this.extent = extent;
@@ -234,6 +243,8 @@
     setEntriesWrittenIsSet(true);
     this.ssiList = ssiList;
     this.ssio = ssio;
+    this.timesPaused = timesPaused;
+    setTimesPausedIsSet(true);
   }
 
   /**
@@ -285,6 +296,7 @@
       }
       this.ssio = __this__ssio;
     }
+    this.timesPaused = other.timesPaused;
   }
 
   @Override
@@ -308,6 +320,8 @@
     this.entriesWritten = 0;
     this.ssiList = null;
     this.ssio = null;
+    setTimesPausedIsSet(false);
+    this.timesPaused = 0;
   }
 
   @org.apache.thrift.annotation.Nullable
@@ -638,6 +652,29 @@
     }
   }
 
+  public long getTimesPaused() {
+    return this.timesPaused;
+  }
+
+  public ActiveCompaction setTimesPaused(long timesPaused) {
+    this.timesPaused = timesPaused;
+    setTimesPausedIsSet(true);
+    return this;
+  }
+
+  public void unsetTimesPaused() {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __TIMESPAUSED_ISSET_ID);
+  }
+
+  /** Returns true if field timesPaused is set (has been assigned a value) and false otherwise */
+  public boolean isSetTimesPaused() {
+    return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __TIMESPAUSED_ISSET_ID);
+  }
+
+  public void setTimesPausedIsSet(boolean value) {
+    __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __TIMESPAUSED_ISSET_ID, value);
+  }
+
   @Override
   public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
     switch (field) {
@@ -729,6 +766,14 @@
       }
       break;
 
+    case TIMES_PAUSED:
+      if (value == null) {
+        unsetTimesPaused();
+      } else {
+        setTimesPaused((java.lang.Long)value);
+      }
+      break;
+
     }
   }
 
@@ -769,6 +814,9 @@
     case SSIO:
       return getSsio();
 
+    case TIMES_PAUSED:
+      return getTimesPaused();
+
     }
     throw new java.lang.IllegalStateException();
   }
@@ -803,6 +851,8 @@
       return isSetSsiList();
     case SSIO:
       return isSetSsio();
+    case TIMES_PAUSED:
+      return isSetTimesPaused();
     }
     throw new java.lang.IllegalStateException();
   }
@@ -919,6 +969,15 @@
         return false;
     }
 
+    boolean this_present_timesPaused = true;
+    boolean that_present_timesPaused = true;
+    if (this_present_timesPaused || that_present_timesPaused) {
+      if (!(this_present_timesPaused && that_present_timesPaused))
+        return false;
+      if (this.timesPaused != that.timesPaused)
+        return false;
+    }
+
     return true;
   }
 
@@ -964,6 +1023,8 @@
     if (isSetSsio())
       hashCode = hashCode * 8191 + ssio.hashCode();
 
+    hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(timesPaused);
+
     return hashCode;
   }
 
@@ -1085,6 +1146,16 @@
         return lastComparison;
       }
     }
+    lastComparison = java.lang.Boolean.compare(isSetTimesPaused(), other.isSetTimesPaused());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTimesPaused()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timesPaused, other.timesPaused);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1184,6 +1255,10 @@
       sb.append(this.ssio);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("timesPaused:");
+    sb.append(this.timesPaused);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -1254,13 +1329,13 @@
           case 3: // INPUT_FILES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list52 = iprot.readListBegin();
-                struct.inputFiles = new java.util.ArrayList<java.lang.String>(_list52.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _elem53;
-                for (int _i54 = 0; _i54 < _list52.size; ++_i54)
+                org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
+                struct.inputFiles = new java.util.ArrayList<java.lang.String>(_list0.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _elem1;
+                for (int _i2 = 0; _i2 < _list0.size; ++_i2)
                 {
-                  _elem53 = iprot.readString();
-                  struct.inputFiles.add(_elem53);
+                  _elem1 = iprot.readString();
+                  struct.inputFiles.add(_elem1);
                 }
                 iprot.readListEnd();
               }
@@ -1320,14 +1395,14 @@
           case 10: // SSI_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list55 = iprot.readListBegin();
-                struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list55.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem56;
-                for (int _i57 = 0; _i57 < _list55.size; ++_i57)
+                org.apache.thrift.protocol.TList _list3 = iprot.readListBegin();
+                struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list3.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem4;
+                for (int _i5 = 0; _i5 < _list3.size; ++_i5)
                 {
-                  _elem56 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-                  _elem56.read(iprot);
-                  struct.ssiList.add(_elem56);
+                  _elem4 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+                  _elem4.read(iprot);
+                  struct.ssiList.add(_elem4);
                 }
                 iprot.readListEnd();
               }
@@ -1339,27 +1414,27 @@
           case 11: // SSIO
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map58 = iprot.readMapBegin();
-                struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map58.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key59;
-                @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val60;
-                for (int _i61 = 0; _i61 < _map58.size; ++_i61)
+                org.apache.thrift.protocol.TMap _map6 = iprot.readMapBegin();
+                struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map6.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key7;
+                @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val8;
+                for (int _i9 = 0; _i9 < _map6.size; ++_i9)
                 {
-                  _key59 = iprot.readString();
+                  _key7 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map62 = iprot.readMapBegin();
-                    _val60 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map62.size);
-                    @org.apache.thrift.annotation.Nullable java.lang.String _key63;
-                    @org.apache.thrift.annotation.Nullable java.lang.String _val64;
-                    for (int _i65 = 0; _i65 < _map62.size; ++_i65)
+                    org.apache.thrift.protocol.TMap _map10 = iprot.readMapBegin();
+                    _val8 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map10.size);
+                    @org.apache.thrift.annotation.Nullable java.lang.String _key11;
+                    @org.apache.thrift.annotation.Nullable java.lang.String _val12;
+                    for (int _i13 = 0; _i13 < _map10.size; ++_i13)
                     {
-                      _key63 = iprot.readString();
-                      _val64 = iprot.readString();
-                      _val60.put(_key63, _val64);
+                      _key11 = iprot.readString();
+                      _val12 = iprot.readString();
+                      _val8.put(_key11, _val12);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.ssio.put(_key59, _val60);
+                  struct.ssio.put(_key7, _val8);
                 }
                 iprot.readMapEnd();
               }
@@ -1368,6 +1443,14 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 12: // TIMES_PAUSED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.timesPaused = iprot.readI64();
+              struct.setTimesPausedIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1396,9 +1479,9 @@
         oprot.writeFieldBegin(INPUT_FILES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.inputFiles.size()));
-          for (java.lang.String _iter66 : struct.inputFiles)
+          for (java.lang.String _iter14 : struct.inputFiles)
           {
-            oprot.writeString(_iter66);
+            oprot.writeString(_iter14);
           }
           oprot.writeListEnd();
         }
@@ -1434,9 +1517,9 @@
         oprot.writeFieldBegin(SSI_LIST_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.ssiList.size()));
-          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter67 : struct.ssiList)
+          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter15 : struct.ssiList)
           {
-            _iter67.write(oprot);
+            _iter15.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1446,15 +1529,15 @@
         oprot.writeFieldBegin(SSIO_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.ssio.size()));
-          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter68 : struct.ssio.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter16 : struct.ssio.entrySet())
           {
-            oprot.writeString(_iter68.getKey());
+            oprot.writeString(_iter16.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter68.getValue().size()));
-              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter69 : _iter68.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, _iter16.getValue().size()));
+              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter17 : _iter16.getValue().entrySet())
               {
-                oprot.writeString(_iter69.getKey());
-                oprot.writeString(_iter69.getValue());
+                oprot.writeString(_iter17.getKey());
+                oprot.writeString(_iter17.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1463,6 +1546,9 @@
         }
         oprot.writeFieldEnd();
       }
+      oprot.writeFieldBegin(TIMES_PAUSED_FIELD_DESC);
+      oprot.writeI64(struct.timesPaused);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1515,7 +1601,10 @@
       if (struct.isSetSsio()) {
         optionals.set(10);
       }
-      oprot.writeBitSet(optionals, 11);
+      if (struct.isSetTimesPaused()) {
+        optionals.set(11);
+      }
+      oprot.writeBitSet(optionals, 12);
       if (struct.isSetExtent()) {
         struct.extent.write(oprot);
       }
@@ -1525,9 +1614,9 @@
       if (struct.isSetInputFiles()) {
         {
           oprot.writeI32(struct.inputFiles.size());
-          for (java.lang.String _iter70 : struct.inputFiles)
+          for (java.lang.String _iter18 : struct.inputFiles)
           {
-            oprot.writeString(_iter70);
+            oprot.writeString(_iter18);
           }
         }
       }
@@ -1552,35 +1641,38 @@
       if (struct.isSetSsiList()) {
         {
           oprot.writeI32(struct.ssiList.size());
-          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter71 : struct.ssiList)
+          for (org.apache.accumulo.core.dataImpl.thrift.IterInfo _iter19 : struct.ssiList)
           {
-            _iter71.write(oprot);
+            _iter19.write(oprot);
           }
         }
       }
       if (struct.isSetSsio()) {
         {
           oprot.writeI32(struct.ssio.size());
-          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter72 : struct.ssio.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.util.Map<java.lang.String,java.lang.String>> _iter20 : struct.ssio.entrySet())
           {
-            oprot.writeString(_iter72.getKey());
+            oprot.writeString(_iter20.getKey());
             {
-              oprot.writeI32(_iter72.getValue().size());
-              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter73 : _iter72.getValue().entrySet())
+              oprot.writeI32(_iter20.getValue().size());
+              for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter21 : _iter20.getValue().entrySet())
               {
-                oprot.writeString(_iter73.getKey());
-                oprot.writeString(_iter73.getValue());
+                oprot.writeString(_iter21.getKey());
+                oprot.writeString(_iter21.getValue());
               }
             }
           }
         }
       }
+      if (struct.isSetTimesPaused()) {
+        oprot.writeI64(struct.timesPaused);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, ActiveCompaction struct) throws org.apache.thrift.TException {
       org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-      java.util.BitSet incoming = iprot.readBitSet(11);
+      java.util.BitSet incoming = iprot.readBitSet(12);
       if (incoming.get(0)) {
         struct.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
         struct.extent.read(iprot);
@@ -1592,13 +1684,13 @@
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list74 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-          struct.inputFiles = new java.util.ArrayList<java.lang.String>(_list74.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _elem75;
-          for (int _i76 = 0; _i76 < _list74.size; ++_i76)
+          org.apache.thrift.protocol.TList _list22 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+          struct.inputFiles = new java.util.ArrayList<java.lang.String>(_list22.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _elem23;
+          for (int _i24 = 0; _i24 < _list22.size; ++_i24)
           {
-            _elem75 = iprot.readString();
-            struct.inputFiles.add(_elem75);
+            _elem23 = iprot.readString();
+            struct.inputFiles.add(_elem23);
           }
         }
         struct.setInputFilesIsSet(true);
@@ -1629,44 +1721,48 @@
       }
       if (incoming.get(9)) {
         {
-          org.apache.thrift.protocol.TList _list77 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list77.size);
-          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem78;
-          for (int _i79 = 0; _i79 < _list77.size; ++_i79)
+          org.apache.thrift.protocol.TList _list25 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.ssiList = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.IterInfo>(_list25.size);
+          @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.IterInfo _elem26;
+          for (int _i27 = 0; _i27 < _list25.size; ++_i27)
           {
-            _elem78 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
-            _elem78.read(iprot);
-            struct.ssiList.add(_elem78);
+            _elem26 = new org.apache.accumulo.core.dataImpl.thrift.IterInfo();
+            _elem26.read(iprot);
+            struct.ssiList.add(_elem26);
           }
         }
         struct.setSsiListIsSet(true);
       }
       if (incoming.get(10)) {
         {
-          org.apache.thrift.protocol.TMap _map80 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
-          struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map80.size);
-          @org.apache.thrift.annotation.Nullable java.lang.String _key81;
-          @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val82;
-          for (int _i83 = 0; _i83 < _map80.size; ++_i83)
+          org.apache.thrift.protocol.TMap _map28 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP); 
+          struct.ssio = new java.util.HashMap<java.lang.String,java.util.Map<java.lang.String,java.lang.String>>(2*_map28.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key29;
+          @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.lang.String> _val30;
+          for (int _i31 = 0; _i31 < _map28.size; ++_i31)
           {
-            _key81 = iprot.readString();
+            _key29 = iprot.readString();
             {
-              org.apache.thrift.protocol.TMap _map84 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-              _val82 = 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 _map32 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+              _val30 = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map32.size);
+              @org.apache.thrift.annotation.Nullable java.lang.String _key33;
+              @org.apache.thrift.annotation.Nullable java.lang.String _val34;
+              for (int _i35 = 0; _i35 < _map32.size; ++_i35)
               {
-                _key85 = iprot.readString();
-                _val86 = iprot.readString();
-                _val82.put(_key85, _val86);
+                _key33 = iprot.readString();
+                _val34 = iprot.readString();
+                _val30.put(_key33, _val34);
               }
             }
-            struct.ssio.put(_key81, _val82);
+            struct.ssio.put(_key29, _val30);
           }
         }
         struct.setSsioIsSet(true);
       }
+      if (incoming.get(11)) {
+        struct.timesPaused = iprot.readI64();
+        struct.setTimesPausedIsSet(true);
+      }
     }
   }
 
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java
index 83ab70d..d03a8d2 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/IteratorConfig.java
@@ -354,14 +354,14 @@
           case 1: // ITERATORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list98 = iprot.readListBegin();
-                struct.iterators = new java.util.ArrayList<TIteratorSetting>(_list98.size);
-                @org.apache.thrift.annotation.Nullable TIteratorSetting _elem99;
-                for (int _i100 = 0; _i100 < _list98.size; ++_i100)
+                org.apache.thrift.protocol.TList _list46 = iprot.readListBegin();
+                struct.iterators = new java.util.ArrayList<TIteratorSetting>(_list46.size);
+                @org.apache.thrift.annotation.Nullable TIteratorSetting _elem47;
+                for (int _i48 = 0; _i48 < _list46.size; ++_i48)
                 {
-                  _elem99 = new TIteratorSetting();
-                  _elem99.read(iprot);
-                  struct.iterators.add(_elem99);
+                  _elem47 = new TIteratorSetting();
+                  _elem47.read(iprot);
+                  struct.iterators.add(_elem47);
                 }
                 iprot.readListEnd();
               }
@@ -390,9 +390,9 @@
         oprot.writeFieldBegin(ITERATORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.iterators.size()));
-          for (TIteratorSetting _iter101 : struct.iterators)
+          for (TIteratorSetting _iter49 : struct.iterators)
           {
-            _iter101.write(oprot);
+            _iter49.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -424,9 +424,9 @@
       if (struct.isSetIterators()) {
         {
           oprot.writeI32(struct.iterators.size());
-          for (TIteratorSetting _iter102 : struct.iterators)
+          for (TIteratorSetting _iter50 : struct.iterators)
           {
-            _iter102.write(oprot);
+            _iter50.write(oprot);
           }
         }
       }
@@ -438,14 +438,14 @@
       java.util.BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list103 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.iterators = new java.util.ArrayList<TIteratorSetting>(_list103.size);
-          @org.apache.thrift.annotation.Nullable TIteratorSetting _elem104;
-          for (int _i105 = 0; _i105 < _list103.size; ++_i105)
+          org.apache.thrift.protocol.TList _list51 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.iterators = new java.util.ArrayList<TIteratorSetting>(_list51.size);
+          @org.apache.thrift.annotation.Nullable TIteratorSetting _elem52;
+          for (int _i53 = 0; _i53 < _list51.size; ++_i53)
           {
-            _elem104 = new TIteratorSetting();
-            _elem104.read(iprot);
-            struct.iterators.add(_elem104);
+            _elem52 = new TIteratorSetting();
+            _elem52.read(iprot);
+            struct.iterators.add(_elem52);
           }
         }
         struct.setIteratorsIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionKind.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionKind.java
index eb0aae3..8600b93 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionKind.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionKind.java
@@ -26,10 +26,9 @@
 
 
 public enum TCompactionKind implements org.apache.thrift.TEnum {
-  CHOP(0),
-  SELECTOR(1),
-  SYSTEM(2),
-  USER(3);
+  SELECTOR(0),
+  SYSTEM(1),
+  USER(2);
 
   private final int value;
 
@@ -53,12 +52,10 @@
   public static TCompactionKind findByValue(int value) { 
     switch (value) {
       case 0:
-        return CHOP;
-      case 1:
         return SELECTOR;
-      case 2:
+      case 1:
         return SYSTEM;
-      case 3:
+      case 2:
         return USER;
       default:
         return null;
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionReason.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionReason.java
index 2c640dc..09c99f0 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionReason.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TCompactionReason.java
@@ -28,9 +28,8 @@
 public enum TCompactionReason implements org.apache.thrift.TEnum {
   USER(0),
   SYSTEM(1),
-  CHOP(2),
-  IDLE(3),
-  CLOSE(4);
+  IDLE(2),
+  CLOSE(3);
 
   private final int value;
 
@@ -58,10 +57,8 @@
       case 1:
         return SYSTEM;
       case 2:
-        return CHOP;
-      case 3:
         return IDLE;
-      case 4:
+      case 3:
         return CLOSE;
       default:
         return null;
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java
index 80cec38..e440c9c 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TExternalCompactionJob.java
@@ -1037,14 +1037,14 @@
           case 3: // FILES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list116 = iprot.readListBegin();
-                struct.files = new java.util.ArrayList<InputFile>(_list116.size);
-                @org.apache.thrift.annotation.Nullable InputFile _elem117;
-                for (int _i118 = 0; _i118 < _list116.size; ++_i118)
+                org.apache.thrift.protocol.TList _list54 = iprot.readListBegin();
+                struct.files = new java.util.ArrayList<InputFile>(_list54.size);
+                @org.apache.thrift.annotation.Nullable InputFile _elem55;
+                for (int _i56 = 0; _i56 < _list54.size; ++_i56)
                 {
-                  _elem117 = new InputFile();
-                  _elem117.read(iprot);
-                  struct.files.add(_elem117);
+                  _elem55 = new InputFile();
+                  _elem55.read(iprot);
+                  struct.files.add(_elem55);
                 }
                 iprot.readListEnd();
               }
@@ -1097,15 +1097,15 @@
           case 9: // OVERRIDES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map119 = iprot.readMapBegin();
-                struct.overrides = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map119.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key120;
-                @org.apache.thrift.annotation.Nullable java.lang.String _val121;
-                for (int _i122 = 0; _i122 < _map119.size; ++_i122)
+                org.apache.thrift.protocol.TMap _map57 = iprot.readMapBegin();
+                struct.overrides = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map57.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key58;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val59;
+                for (int _i60 = 0; _i60 < _map57.size; ++_i60)
                 {
-                  _key120 = iprot.readString();
-                  _val121 = iprot.readString();
-                  struct.overrides.put(_key120, _val121);
+                  _key58 = iprot.readString();
+                  _val59 = iprot.readString();
+                  struct.overrides.put(_key58, _val59);
                 }
                 iprot.readMapEnd();
               }
@@ -1144,9 +1144,9 @@
         oprot.writeFieldBegin(FILES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.files.size()));
-          for (InputFile _iter123 : struct.files)
+          for (InputFile _iter61 : struct.files)
           {
-            _iter123.write(oprot);
+            _iter61.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1177,10 +1177,10 @@
         oprot.writeFieldBegin(OVERRIDES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.overrides.size()));
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter124 : struct.overrides.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter62 : struct.overrides.entrySet())
           {
-            oprot.writeString(_iter124.getKey());
-            oprot.writeString(_iter124.getValue());
+            oprot.writeString(_iter62.getKey());
+            oprot.writeString(_iter62.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1242,9 +1242,9 @@
       if (struct.isSetFiles()) {
         {
           oprot.writeI32(struct.files.size());
-          for (InputFile _iter125 : struct.files)
+          for (InputFile _iter63 : struct.files)
           {
-            _iter125.write(oprot);
+            _iter63.write(oprot);
           }
         }
       }
@@ -1266,10 +1266,10 @@
       if (struct.isSetOverrides()) {
         {
           oprot.writeI32(struct.overrides.size());
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter126 : struct.overrides.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter64 : struct.overrides.entrySet())
           {
-            oprot.writeString(_iter126.getKey());
-            oprot.writeString(_iter126.getValue());
+            oprot.writeString(_iter64.getKey());
+            oprot.writeString(_iter64.getValue());
           }
         }
       }
@@ -1290,14 +1290,14 @@
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list127 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-          struct.files = new java.util.ArrayList<InputFile>(_list127.size);
-          @org.apache.thrift.annotation.Nullable InputFile _elem128;
-          for (int _i129 = 0; _i129 < _list127.size; ++_i129)
+          org.apache.thrift.protocol.TList _list65 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+          struct.files = new java.util.ArrayList<InputFile>(_list65.size);
+          @org.apache.thrift.annotation.Nullable InputFile _elem66;
+          for (int _i67 = 0; _i67 < _list65.size; ++_i67)
           {
-            _elem128 = new InputFile();
-            _elem128.read(iprot);
-            struct.files.add(_elem128);
+            _elem66 = new InputFile();
+            _elem66.read(iprot);
+            struct.files.add(_elem66);
           }
         }
         struct.setFilesIsSet(true);
@@ -1325,15 +1325,15 @@
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TMap _map130 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-          struct.overrides = 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 _map68 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+          struct.overrides = 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)
           {
-            _key131 = iprot.readString();
-            _val132 = iprot.readString();
-            struct.overrides.put(_key131, _val132);
+            _key69 = iprot.readString();
+            _val70 = iprot.readString();
+            struct.overrides.put(_key69, _val70);
           }
         }
         struct.setOverridesIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java
index a58faa4..cbf0841 100644
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TIteratorSetting.java
@@ -614,15 +614,15 @@
           case 4: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map88 = iprot.readMapBegin();
-                struct.properties = 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)
+                org.apache.thrift.protocol.TMap _map36 = iprot.readMapBegin();
+                struct.properties = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map36.size);
+                @org.apache.thrift.annotation.Nullable java.lang.String _key37;
+                @org.apache.thrift.annotation.Nullable java.lang.String _val38;
+                for (int _i39 = 0; _i39 < _map36.size; ++_i39)
                 {
-                  _key89 = iprot.readString();
-                  _val90 = iprot.readString();
-                  struct.properties.put(_key89, _val90);
+                  _key37 = iprot.readString();
+                  _val38 = iprot.readString();
+                  struct.properties.put(_key37, _val38);
                 }
                 iprot.readMapEnd();
               }
@@ -664,10 +664,10 @@
         oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter92 : struct.properties.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter40 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter92.getKey());
-            oprot.writeString(_iter92.getValue());
+            oprot.writeString(_iter40.getKey());
+            oprot.writeString(_iter40.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -717,10 +717,10 @@
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter93 : struct.properties.entrySet())
+          for (java.util.Map.Entry<java.lang.String, java.lang.String> _iter41 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter93.getKey());
-            oprot.writeString(_iter93.getValue());
+            oprot.writeString(_iter41.getKey());
+            oprot.writeString(_iter41.getValue());
           }
         }
       }
@@ -744,15 +744,15 @@
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
-          struct.properties = 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 _map42 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING); 
+          struct.properties = new java.util.HashMap<java.lang.String,java.lang.String>(2*_map42.size);
+          @org.apache.thrift.annotation.Nullable java.lang.String _key43;
+          @org.apache.thrift.annotation.Nullable java.lang.String _val44;
+          for (int _i45 = 0; _i45 < _map42.size; ++_i45)
           {
-            _key95 = iprot.readString();
-            _val96 = iprot.readString();
-            struct.properties.put(_key95, _val96);
+            _key43 = iprot.readString();
+            _val44 = iprot.readString();
+            struct.properties.put(_key43, _val44);
           }
         }
         struct.setPropertiesIsSet(true);
diff --git a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
deleted file mode 100644
index 8f734f2..0000000
--- a/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ /dev/null
@@ -1,39560 +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.tabletserver.thrift;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-public class TabletClientService {
-
-  public interface Iface {
-
-    public long startUpdate(org.apache.accumulo.core.trace.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.trace.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.trace.thrift.TInfo tinfo, long updateID) throws NoSuchScanIDException, org.apache.thrift.TException;
-
-    public boolean cancelUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long updateID) throws org.apache.thrift.TException;
-
-    public void update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.core.trace.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.trace.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 NoSuchScanIDException, org.apache.thrift.TException;
-
-    public void invalidateConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
-
-    public void closeConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException;
-
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public void loadFiles(org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.thrift.TException;
-
-    public void splitTablet(org.apache.accumulo.core.trace.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, NotServingTabletException, org.apache.thrift.TException;
-
-    public void loadTablet(org.apache.accumulo.core.trace.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.trace.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 flush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException;
-
-    public void flushTablet(org.apache.accumulo.core.trace.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 chop(org.apache.accumulo.core.trace.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 compact(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.master.thrift.TabletServerStatus getTabletServerStatus(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<TabletStats> getTabletStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public TabletStats getHistoricalStats(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 void halt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public void fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException;
-
-    public java.util.List<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 void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException;
-
-    public java.util.List<java.lang.String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException;
-
-    public java.util.List<TCompactionQueueSummary> getCompactionQueueInfo(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 TExternalCompactionJob reserveCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
-
-    public void compactionJobFinished(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, long fileSize, long entries) throws org.apache.thrift.TException;
-
-    public void compactionJobFailed(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 interface AsyncIface {
-
-    public void startUpdate(org.apache.accumulo.core.trace.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.trace.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.trace.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.trace.thrift.TInfo tinfo, long updateID, org.apache.thrift.async.AsyncMethodCallback<java.lang.Boolean> resultHandler) throws org.apache.thrift.TException;
-
-    public void update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void startConditionalUpdate(org.apache.accumulo.core.trace.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.trace.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.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void closeConditionalUpdate(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessID, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> resultHandler) throws org.apache.thrift.TException;
-
-    public void loadFiles(org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void splitTablet(org.apache.accumulo.core.trace.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 loadTablet(org.apache.accumulo.core.trace.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.trace.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 flush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void flushTablet(org.apache.accumulo.core.trace.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 chop(org.apache.accumulo.core.trace.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 compact(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void getTabletServerStatus(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.master.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException;
-
-    public void getTabletStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException;
-
-    public void getHistoricalStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException;
-
-    public void halt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> 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<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
-
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void getActiveLogs(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 startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
-
-    public void startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
-
-    public void startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
-
-    public void contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
-
-    public void getCompactionQueueInfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException;
-
-    public void reserveCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
-
-    public void compactionJobFinished(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, long fileSize, long entries, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
-
-    public void compactionJobFailed(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 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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo tinfo, long updateID) throws NoSuchScanIDException, org.apache.thrift.TException
-    {
-      send_closeUpdate(tinfo, updateID);
-      return recv_closeUpdate();
-    }
-
-    public void send_closeUpdate(org.apache.accumulo.core.trace.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 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.trace.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.trace.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 void update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException
-    {
-      send_update(tinfo, credentials, keyExtent, mutation, durability);
-      recv_update();
-    }
-
-    public void send_update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability) throws org.apache.thrift.TException
-    {
-      update_args args = new update_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setKeyExtent(keyExtent);
-      args.setMutation(mutation);
-      args.setDurability(durability);
-      sendBase("update", args);
-    }
-
-    public void recv_update() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, org.apache.thrift.TException
-    {
-      update_result result = new update_result();
-      receiveBase(result, "update");
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      if (result.nste != null) {
-        throw result.nste;
-      }
-      if (result.cve != null) {
-        throw result.cve;
-      }
-      return;
-    }
-
-    @Override
-    public org.apache.accumulo.core.dataImpl.thrift.TConditionalSession startConditionalUpdate(org.apache.accumulo.core.trace.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.trace.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.trace.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 NoSuchScanIDException, org.apache.thrift.TException
-    {
-      send_conditionalUpdate(tinfo, sessID, mutations, symbols);
-      return recv_conditionalUpdate();
-    }
-
-    public void send_conditionalUpdate(org.apache.accumulo.core.trace.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 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.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
-    {
-      send_invalidateConditionalUpdate(tinfo, sessID);
-      recv_invalidateConditionalUpdate();
-    }
-
-    public void send_invalidateConditionalUpdate(org.apache.accumulo.core.trace.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.trace.thrift.TInfo tinfo, long sessID) throws org.apache.thrift.TException
-    {
-      send_closeConditionalUpdate(tinfo, sessID);
-    }
-
-    public void send_closeConditionalUpdate(org.apache.accumulo.core.trace.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 java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_bulkImport(tinfo, credentials, tid, files, setTime);
-      return recv_bulkImport();
-    }
-
-    public void send_bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
-    {
-      bulkImport_args args = new bulkImport_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setTid(tid);
-      args.setFiles(files);
-      args.setSetTime(setTime);
-      sendBase("bulkImport", args);
-    }
-
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> recv_bulkImport() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      bulkImport_result result = new bulkImport_result();
-      receiveBase(result, "bulkImport");
-      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, "bulkImport failed: unknown result");
-    }
-
-    @Override
-    public void loadFiles(org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime) throws org.apache.thrift.TException
-    {
-      send_loadFiles(tinfo, credentials, tid, dir, files, setTime);
-    }
-
-    public void send_loadFiles(org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> 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);
-    }
-
-    @Override
-    public void splitTablet(org.apache.accumulo.core.trace.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, NotServingTabletException, org.apache.thrift.TException
-    {
-      send_splitTablet(tinfo, credentials, extent, splitPoint);
-      recv_splitTablet();
-    }
-
-    public void send_splitTablet(org.apache.accumulo.core.trace.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, 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 loadTablet(org.apache.accumulo.core.trace.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.trace.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.trace.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.trace.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 flush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
-    {
-      send_flush(tinfo, credentials, lock, tableId, startRow, endRow);
-    }
-
-    public void send_flush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
-    {
-      flush_args args = new flush_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setLock(lock);
-      args.setTableId(tableId);
-      args.setStartRow(startRow);
-      args.setEndRow(endRow);
-      sendBaseOneway("flush", args);
-    }
-
-    @Override
-    public void flushTablet(org.apache.accumulo.core.trace.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.trace.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);
-    }
-
-    @Override
-    public void chop(org.apache.accumulo.core.trace.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_chop(tinfo, credentials, lock, extent);
-    }
-
-    public void send_chop(org.apache.accumulo.core.trace.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
-    {
-      chop_args args = new chop_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setLock(lock);
-      args.setExtent(extent);
-      sendBaseOneway("chop", args);
-    }
-
-    @Override
-    public void compact(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
-    {
-      send_compact(tinfo, credentials, lock, tableId, startRow, endRow);
-    }
-
-    public void send_compact(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
-    {
-      compact_args args = new compact_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setLock(lock);
-      args.setTableId(tableId);
-      args.setStartRow(startRow);
-      args.setEndRow(endRow);
-      sendBaseOneway("compact", args);
-    }
-
-    @Override
-    public org.apache.accumulo.core.master.thrift.TabletServerStatus getTabletServerStatus(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
-    {
-      send_getTabletServerStatus(tinfo, credentials);
-      return recv_getTabletServerStatus();
-    }
-
-    public void send_getTabletServerStatus(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      getTabletServerStatus_args args = new getTabletServerStatus_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("getTabletServerStatus", args);
-    }
-
-    public org.apache.accumulo.core.master.thrift.TabletServerStatus recv_getTabletServerStatus() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      getTabletServerStatus_result result = new getTabletServerStatus_result();
-      receiveBase(result, "getTabletServerStatus");
-      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, "getTabletServerStatus failed: unknown result");
-    }
-
-    @Override
-    public java.util.List<TabletStats> getTabletStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_getTabletStats(tinfo, credentials, tableId);
-      return recv_getTabletStats();
-    }
-
-    public void send_getTabletStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId) throws org.apache.thrift.TException
-    {
-      getTabletStats_args args = new getTabletStats_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setTableId(tableId);
-      sendBase("getTabletStats", args);
-    }
-
-    public java.util.List<TabletStats> recv_getTabletStats() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      getTabletStats_result result = new getTabletStats_result();
-      receiveBase(result, "getTabletStats");
-      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, "getTabletStats failed: unknown result");
-    }
-
-    @Override
-    public TabletStats getHistoricalStats(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
-    {
-      send_getHistoricalStats(tinfo, credentials);
-      return recv_getHistoricalStats();
-    }
-
-    public void send_getHistoricalStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      getHistoricalStats_args args = new getHistoricalStats_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("getHistoricalStats", args);
-    }
-
-    public TabletStats recv_getHistoricalStats() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      getHistoricalStats_result result = new getHistoricalStats_result();
-      receiveBase(result, "getHistoricalStats");
-      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, "getHistoricalStats failed: unknown result");
-    }
-
-    @Override
-    public void halt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_halt(tinfo, credentials, lock);
-      recv_halt();
-    }
-
-    public void send_halt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException
-    {
-      halt_args args = new halt_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setLock(lock);
-      sendBase("halt", args);
-    }
-
-    public void recv_halt() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      halt_result result = new halt_result();
-      receiveBase(result, "halt");
-      if (result.sec != null) {
-        throw result.sec;
-      }
-      return;
-    }
-
-    @Override
-    public void fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException
-    {
-      send_fastHalt(tinfo, credentials, lock);
-    }
-
-    public void send_fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException
-    {
-      fastHalt_args args = new fastHalt_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setLock(lock);
-      sendBaseOneway("fastHalt", args);
-    }
-
-    @Override
-    public java.util.List<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
-    {
-      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
-    {
-      getActiveCompactions_args args = new getActiveCompactions_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("getActiveCompactions", args);
-    }
-
-    public java.util.List<ActiveCompaction> recv_getActiveCompactions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      getActiveCompactions_result result = new getActiveCompactions_result();
-      receiveBase(result, "getActiveCompactions");
-      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, "getActiveCompactions failed: unknown result");
-    }
-
-    @Override
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException
-    {
-      send_removeLogs(tinfo, credentials, filenames);
-    }
-
-    public void send_removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException
-    {
-      removeLogs_args args = new removeLogs_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setFilenames(filenames);
-      sendBaseOneway("removeLogs", args);
-    }
-
-    @Override
-    public java.util.List<java.lang.String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      send_getActiveLogs(tinfo, credentials);
-      return recv_getActiveLogs();
-    }
-
-    public void send_getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      getActiveLogs_args args = new getActiveLogs_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("getActiveLogs", args);
-    }
-
-    public java.util.List<java.lang.String> recv_getActiveLogs() throws org.apache.thrift.TException
-    {
-      getActiveLogs_result result = new getActiveLogs_result();
-      receiveBase(result, "getActiveLogs");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveLogs failed: unknown result");
-    }
-
-    @Override
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      send_startGetSummaries(tinfo, credentials, request);
-      return recv_startGetSummaries();
-    }
-
-    public void send_startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) throws org.apache.thrift.TException
-    {
-      startGetSummaries_args args = new startGetSummaries_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setRequest(request);
-      sendBase("startGetSummaries", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_startGetSummaries() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException
-    {
-      startGetSummaries_result result = new startGetSummaries_result();
-      receiveBase(result, "startGetSummaries");
-      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, "startGetSummaries failed: unknown result");
-    }
-
-    @Override
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_startGetSummariesForPartition(tinfo, credentials, request, modulus, remainder);
-      return recv_startGetSummariesForPartition();
-    }
-
-    public void send_startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.thrift.TException
-    {
-      startGetSummariesForPartition_args args = new startGetSummariesForPartition_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setRequest(request);
-      args.setModulus(modulus);
-      args.setRemainder(remainder);
-      sendBase("startGetSummariesForPartition", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_startGetSummariesForPartition() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
-      receiveBase(result, "startGetSummariesForPartition");
-      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, "startGetSummariesForPartition failed: unknown result");
-    }
-
-    @Override
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_startGetSummariesFromFiles(tinfo, credentials, request, files);
-      return recv_startGetSummariesFromFiles();
-    }
-
-    public void send_startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files) throws org.apache.thrift.TException
-    {
-      startGetSummariesFromFiles_args args = new startGetSummariesFromFiles_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setRequest(request);
-      args.setFiles(files);
-      sendBase("startGetSummariesFromFiles", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_startGetSummariesFromFiles() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
-      receiveBase(result, "startGetSummariesFromFiles");
-      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, "startGetSummariesFromFiles failed: unknown result");
-    }
-
-    @Override
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException
-    {
-      send_contiuneGetSummaries(tinfo, sessionId);
-      return recv_contiuneGetSummaries();
-    }
-
-    public void send_contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId) throws org.apache.thrift.TException
-    {
-      contiuneGetSummaries_args args = new contiuneGetSummaries_args();
-      args.setTinfo(tinfo);
-      args.setSessionId(sessionId);
-      sendBase("contiuneGetSummaries", args);
-    }
-
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_contiuneGetSummaries() throws NoSuchScanIDException, org.apache.thrift.TException
-    {
-      contiuneGetSummaries_result result = new contiuneGetSummaries_result();
-      receiveBase(result, "contiuneGetSummaries");
-      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, "contiuneGetSummaries failed: unknown result");
-    }
-
-    @Override
-    public java.util.List<TCompactionQueueSummary> getCompactionQueueInfo(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
-    {
-      send_getCompactionQueueInfo(tinfo, credentials);
-      return recv_getCompactionQueueInfo();
-    }
-
-    public void send_getCompactionQueueInfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
-    {
-      getCompactionQueueInfo_args args = new getCompactionQueueInfo_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      sendBase("getCompactionQueueInfo", args);
-    }
-
-    public java.util.List<TCompactionQueueSummary> recv_getCompactionQueueInfo() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
-      receiveBase(result, "getCompactionQueueInfo");
-      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, "getCompactionQueueInfo failed: unknown result");
-    }
-
-    @Override
-    public TExternalCompactionJob reserveCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      send_reserveCompactionJob(tinfo, credentials, queueName, priority, compactor, externalCompactionId);
-      return recv_reserveCompactionJob();
-    }
-
-    public void send_reserveCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
-    {
-      reserveCompactionJob_args args = new reserveCompactionJob_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setQueueName(queueName);
-      args.setPriority(priority);
-      args.setCompactor(compactor);
-      args.setExternalCompactionId(externalCompactionId);
-      sendBase("reserveCompactionJob", args);
-    }
-
-    public TExternalCompactionJob recv_reserveCompactionJob() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
-    {
-      reserveCompactionJob_result result = new reserveCompactionJob_result();
-      receiveBase(result, "reserveCompactionJob");
-      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, "reserveCompactionJob failed: unknown result");
-    }
-
-    @Override
-    public void compactionJobFinished(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, long fileSize, long entries) throws org.apache.thrift.TException
-    {
-      send_compactionJobFinished(tinfo, credentials, externalCompactionId, extent, fileSize, entries);
-    }
-
-    public void send_compactionJobFinished(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, long fileSize, long entries) throws org.apache.thrift.TException
-    {
-      compactionJobFinished_args args = new compactionJobFinished_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setExternalCompactionId(externalCompactionId);
-      args.setExtent(extent);
-      args.setFileSize(fileSize);
-      args.setEntries(entries);
-      sendBaseOneway("compactionJobFinished", args);
-    }
-
-    @Override
-    public void compactionJobFailed(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
-    {
-      send_compactionJobFailed(tinfo, credentials, externalCompactionId, extent);
-    }
-
-    public void send_compactionJobFailed(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
-    {
-      compactionJobFailed_args args = new compactionJobFailed_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setExternalCompactionId(externalCompactionId);
-      args.setExtent(extent);
-      sendBaseOneway("compactionJobFailed", 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.trace.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.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private TDurability durability;
-      public startUpdate_call(org.apache.accumulo.core.trace.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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo tinfo;
-      private long updateID;
-      public closeUpdate_call(org.apache.accumulo.core.trace.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 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.trace.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.trace.thrift.TInfo tinfo;
-      private long updateID;
-      public cancelUpdate_call(org.apache.accumulo.core.trace.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 update(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      update_call method_call = new update_call(tinfo, credentials, keyExtent, mutation, durability, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class update_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent;
-      private org.apache.accumulo.core.dataImpl.thrift.TMutation mutation;
-      private TDurability durability;
-      public update_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent, org.apache.accumulo.core.dataImpl.thrift.TMutation mutation, TDurability durability, 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.keyExtent = keyExtent;
-        this.mutation = mutation;
-        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("update", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        update_args args = new update_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setKeyExtent(keyExtent);
-        args.setMutation(mutation);
-        args.setDurability(durability);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public Void getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, NotServingTabletException, ConstraintViolationException, 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_update();
-        return null;
-      }
-    }
-
-    @Override
-    public void startConditionalUpdate(org.apache.accumulo.core.trace.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.trace.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.trace.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.trace.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.trace.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.trace.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 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.trace.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.trace.thrift.TInfo tinfo;
-      private long sessID;
-      public invalidateConditionalUpdate_call(org.apache.accumulo.core.trace.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.trace.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.trace.thrift.TInfo tinfo;
-      private long sessID;
-      public closeConditionalUpdate_call(org.apache.accumulo.core.trace.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 bulkImport(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      bulkImport_call method_call = new bulkImport_call(tinfo, credentials, tid, files, setTime, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class bulkImport_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private long tid;
-      private java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files;
-      private boolean setTime;
-      public bulkImport_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, long tid, java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files, boolean setTime, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> 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.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("bulkImport", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        bulkImport_args args = new bulkImport_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setTid(tid);
-        args.setFiles(files);
-        args.setSetTime(setTime);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> 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_bulkImport();
-      }
-    }
-
-    @Override
-    public void loadFiles(org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> 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.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files;
-      private boolean setTime;
-      public loadFiles_call(org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> 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;
-      }
-    }
-
-    @Override
-    public void splitTablet(org.apache.accumulo.core.trace.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.trace.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.trace.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, 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 loadTablet(org.apache.accumulo.core.trace.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.trace.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.trace.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.trace.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.trace.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.trace.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 flush(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      flush_call method_call = new flush_call(tinfo, credentials, lock, tableId, startRow, endRow, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class flush_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String lock;
-      private java.lang.String tableId;
-      private java.nio.ByteBuffer startRow;
-      private java.nio.ByteBuffer endRow;
-      public flush_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, 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.tableId = tableId;
-        this.startRow = startRow;
-        this.endRow = endRow;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("flush", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        flush_args args = new flush_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setLock(lock);
-        args.setTableId(tableId);
-        args.setStartRow(startRow);
-        args.setEndRow(endRow);
-        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 flushTablet(org.apache.accumulo.core.trace.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.trace.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.trace.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;
-      }
-    }
-
-    @Override
-    public void chop(org.apache.accumulo.core.trace.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();
-      chop_call method_call = new chop_call(tinfo, credentials, lock, extent, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class chop_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.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 chop_call(org.apache.accumulo.core.trace.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("chop", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        chop_args args = new chop_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 compact(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      compact_call method_call = new compact_call(tinfo, credentials, lock, tableId, startRow, endRow, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class compact_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String lock;
-      private java.lang.String tableId;
-      private java.nio.ByteBuffer startRow;
-      private java.nio.ByteBuffer endRow;
-      public compact_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, 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.tableId = tableId;
-        this.startRow = startRow;
-        this.endRow = endRow;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compact", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        compact_args args = new compact_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setLock(lock);
-        args.setTableId(tableId);
-        args.setStartRow(startRow);
-        args.setEndRow(endRow);
-        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 getTabletServerStatus(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.master.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getTabletServerStatus_call method_call = new getTabletServerStatus_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getTabletServerStatus_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.master.thrift.TabletServerStatus> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getTabletServerStatus_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.master.thrift.TabletServerStatus> 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;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTabletServerStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTabletServerStatus_args args = new getTabletServerStatus_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public org.apache.accumulo.core.master.thrift.TabletServerStatus 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_getTabletServerStatus();
-      }
-    }
-
-    @Override
-    public void getTabletStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getTabletStats_call method_call = new getTabletStats_call(tinfo, credentials, tableId, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getTabletStats_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<TabletStats>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String tableId;
-      public getTabletStats_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> 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.tableId = tableId;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTabletStats", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTabletStats_args args = new getTabletStats_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setTableId(tableId);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.util.List<TabletStats> 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_getTabletStats();
-      }
-    }
-
-    @Override
-    public void getHistoricalStats(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getHistoricalStats_call method_call = new getHistoricalStats_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getHistoricalStats_call extends org.apache.thrift.async.TAsyncMethodCall<TabletStats> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getHistoricalStats_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TabletStats> 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;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getHistoricalStats", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getHistoricalStats_args args = new getHistoricalStats_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public TabletStats 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_getHistoricalStats();
-      }
-    }
-
-    @Override
-    public void halt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      halt_call method_call = new halt_call(tinfo, credentials, lock, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class halt_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String lock;
-      public halt_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, 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.lock = lock;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("halt", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        halt_args args = new halt_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setLock(lock);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public Void 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);
-        (new Client(prot)).recv_halt();
-        return null;
-      }
-    }
-
-    @Override
-    public void fastHalt(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      fastHalt_call method_call = new fastHalt_call(tinfo, credentials, lock, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class fastHalt_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String lock;
-      public fastHalt_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, 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;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fastHalt", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        fastHalt_args args = new fastHalt_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setLock(lock);
-        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 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<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;
-      ___manager.call(method_call);
-    }
-
-    public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<ActiveCompaction>> {
-      private org.apache.accumulo.core.trace.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<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;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getActiveCompactions_args args = new getActiveCompactions_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.util.List<ActiveCompaction> 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_getActiveCompactions();
-      }
-    }
-
-    @Override
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      removeLogs_call method_call = new removeLogs_call(tinfo, credentials, filenames, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class removeLogs_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.util.List<java.lang.String> filenames;
-      public removeLogs_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, 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.filenames = filenames;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("removeLogs", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        removeLogs_args args = new removeLogs_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setFilenames(filenames);
-        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 getActiveLogs(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 {
-      checkReady();
-      getActiveLogs_call method_call = new getActiveLogs_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getActiveLogs_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;
-      public getActiveLogs_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 {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.tinfo = tinfo;
-        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("getActiveLogs", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getActiveLogs_args args = new getActiveLogs_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.util.List<java.lang.String> getResult() throws org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new java.lang.IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getActiveLogs();
-      }
-    }
-
-    @Override
-    public void startGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      startGetSummaries_call method_call = new startGetSummaries_call(tinfo, credentials, request, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class startGetSummaries_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request;
-      public startGetSummaries_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.request = request;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummaries", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startGetSummaries_args args = new startGetSummaries_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setRequest(request);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public org.apache.accumulo.core.dataImpl.thrift.TSummaries getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.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_startGetSummaries();
-      }
-    }
-
-    @Override
-    public void startGetSummariesForPartition(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      startGetSummariesForPartition_call method_call = new startGetSummariesForPartition_call(tinfo, credentials, request, modulus, remainder, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class startGetSummariesForPartition_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request;
-      private int modulus;
-      private int remainder;
-      public startGetSummariesForPartition_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.request = request;
-        this.modulus = modulus;
-        this.remainder = remainder;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummariesForPartition", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startGetSummariesForPartition_args args = new startGetSummariesForPartition_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setRequest(request);
-        args.setModulus(modulus);
-        args.setRemainder(remainder);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public org.apache.accumulo.core.dataImpl.thrift.TSummaries 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_startGetSummariesForPartition();
-      }
-    }
-
-    @Override
-    public void startGetSummariesFromFiles(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      startGetSummariesFromFiles_call method_call = new startGetSummariesFromFiles_call(tinfo, credentials, request, files, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class startGetSummariesFromFiles_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request;
-      private java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files;
-      public startGetSummariesFromFiles_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.request = request;
-        this.files = files;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummariesFromFiles", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        startGetSummariesFromFiles_args args = new startGetSummariesFromFiles_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setRequest(request);
-        args.setFiles(files);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public org.apache.accumulo.core.dataImpl.thrift.TSummaries 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_startGetSummariesFromFiles();
-      }
-    }
-
-    @Override
-    public void contiuneGetSummaries(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      contiuneGetSummaries_call method_call = new contiuneGetSummaries_call(tinfo, sessionId, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class contiuneGetSummaries_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private long sessionId;
-      public contiuneGetSummaries_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.sessionId = sessionId;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("contiuneGetSummaries", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        contiuneGetSummaries_args args = new contiuneGetSummaries_args();
-        args.setTinfo(tinfo);
-        args.setSessionId(sessionId);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public org.apache.accumulo.core.dataImpl.thrift.TSummaries getResult() throws 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_contiuneGetSummaries();
-      }
-    }
-
-    @Override
-    public void getCompactionQueueInfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getCompactionQueueInfo_call method_call = new getCompactionQueueInfo_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getCompactionQueueInfo_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<TCompactionQueueSummary>> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      public getCompactionQueueInfo_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<TCompactionQueueSummary>> 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;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionQueueInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getCompactionQueueInfo_args args = new getCompactionQueueInfo_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public java.util.List<TCompactionQueueSummary> 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_getCompactionQueueInfo();
-      }
-    }
-
-    @Override
-    public void reserveCompactionJob(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      reserveCompactionJob_call method_call = new reserveCompactionJob_call(tinfo, credentials, queueName, priority, compactor, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class reserveCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<TExternalCompactionJob> {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
-      private java.lang.String queueName;
-      private long priority;
-      private java.lang.String compactor;
-      private java.lang.String externalCompactionId;
-      public reserveCompactionJob_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<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;
-        this.queueName = queueName;
-        this.priority = priority;
-        this.compactor = compactor;
-        this.externalCompactionId = externalCompactionId;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("reserveCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        reserveCompactionJob_args args = new reserveCompactionJob_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setQueueName(queueName);
-        args.setPriority(priority);
-        args.setCompactor(compactor);
-        args.setExternalCompactionId(externalCompactionId);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      @Override
-      public TExternalCompactionJob 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_reserveCompactionJob();
-      }
-    }
-
-    @Override
-    public void compactionJobFinished(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, long fileSize, long entries, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      compactionJobFinished_call method_call = new compactionJobFinished_call(tinfo, credentials, externalCompactionId, extent, fileSize, entries, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class compactionJobFinished_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.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 long fileSize;
-      private long entries;
-      public compactionJobFinished_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, long fileSize, long entries, 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.externalCompactionId = externalCompactionId;
-        this.extent = extent;
-        this.fileSize = fileSize;
-        this.entries = entries;
-      }
-
-      @Override
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionJobFinished", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        compactionJobFinished_args args = new compactionJobFinished_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setExternalCompactionId(externalCompactionId);
-        args.setExtent(extent);
-        args.setFileSize(fileSize);
-        args.setEntries(entries);
-        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 compactionJobFailed(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 {
-      checkReady();
-      compactionJobFailed_call method_call = new compactionJobFailed_call(tinfo, credentials, externalCompactionId, extent, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class compactionJobFailed_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
-      private org.apache.accumulo.core.trace.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 compactionJobFailed_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 {
-        super(client, protocolFactory, transport, resultHandler, true);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.externalCompactionId = externalCompactionId;
-        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("compactionJobFailed", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
-        compactionJobFailed_args args = new compactionJobFailed_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setExternalCompactionId(externalCompactionId);
-        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("startUpdate", new startUpdate());
-      processMap.put("applyUpdates", new applyUpdates());
-      processMap.put("closeUpdate", new closeUpdate());
-      processMap.put("cancelUpdate", new cancelUpdate());
-      processMap.put("update", new update());
-      processMap.put("startConditionalUpdate", new startConditionalUpdate());
-      processMap.put("conditionalUpdate", new conditionalUpdate());
-      processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
-      processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
-      processMap.put("bulkImport", new bulkImport());
-      processMap.put("loadFiles", new loadFiles());
-      processMap.put("splitTablet", new splitTablet());
-      processMap.put("loadTablet", new loadTablet());
-      processMap.put("unloadTablet", new unloadTablet());
-      processMap.put("flush", new flush());
-      processMap.put("flushTablet", new flushTablet());
-      processMap.put("chop", new chop());
-      processMap.put("compact", new compact());
-      processMap.put("getTabletServerStatus", new getTabletServerStatus());
-      processMap.put("getTabletStats", new getTabletStats());
-      processMap.put("getHistoricalStats", new getHistoricalStats());
-      processMap.put("halt", new halt());
-      processMap.put("fastHalt", new fastHalt());
-      processMap.put("getActiveCompactions", new getActiveCompactions());
-      processMap.put("removeLogs", new removeLogs());
-      processMap.put("getActiveLogs", new getActiveLogs());
-      processMap.put("startGetSummaries", new startGetSummaries());
-      processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
-      processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
-      processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
-      processMap.put("getCompactionQueueInfo", new getCompactionQueueInfo());
-      processMap.put("reserveCompactionJob", new reserveCompactionJob());
-      processMap.put("compactionJobFinished", new compactionJobFinished());
-      processMap.put("compactionJobFailed", new compactionJobFailed());
-      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 (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 update<I extends Iface> extends org.apache.thrift.ProcessFunction<I, update_args> {
-      public update() {
-        super("update");
-      }
-
-      @Override
-      public update_args getEmptyArgsInstance() {
-        return new update_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public update_result getResult(I iface, update_args args) throws org.apache.thrift.TException {
-        update_result result = new update_result();
-        try {
-          iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation, args.durability);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (NotServingTabletException nste) {
-          result.nste = nste;
-        } catch (ConstraintViolationException cve) {
-          result.cve = cve;
-        }
-        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 (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 bulkImport<I extends Iface> extends org.apache.thrift.ProcessFunction<I, bulkImport_args> {
-      public bulkImport() {
-        super("bulkImport");
-      }
-
-      @Override
-      public bulkImport_args getEmptyArgsInstance() {
-        return new bulkImport_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public bulkImport_result getResult(I iface, bulkImport_args args) throws org.apache.thrift.TException {
-        bulkImport_result result = new bulkImport_result();
-        try {
-          result.success = iface.bulkImport(args.tinfo, args.credentials, args.tid, args.files, args.setTime);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    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 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 (NotServingTabletException nste) {
-          result.nste = nste;
-        }
-        return result;
-      }
-    }
-
-    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 flush<I extends Iface> extends org.apache.thrift.ProcessFunction<I, flush_args> {
-      public flush() {
-        super("flush");
-      }
-
-      @Override
-      public flush_args getEmptyArgsInstance() {
-        return new flush_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, flush_args args) throws org.apache.thrift.TException {
-        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow);
-        return null;
-      }
-    }
-
-    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 chop<I extends Iface> extends org.apache.thrift.ProcessFunction<I, chop_args> {
-      public chop() {
-        super("chop");
-      }
-
-      @Override
-      public chop_args getEmptyArgsInstance() {
-        return new chop_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, chop_args args) throws org.apache.thrift.TException {
-        iface.chop(args.tinfo, args.credentials, args.lock, args.extent);
-        return null;
-      }
-    }
-
-    public static class compact<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compact_args> {
-      public compact() {
-        super("compact");
-      }
-
-      @Override
-      public compact_args getEmptyArgsInstance() {
-        return new compact_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, compact_args args) throws org.apache.thrift.TException {
-        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow);
-        return null;
-      }
-    }
-
-    public static class getTabletServerStatus<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTabletServerStatus_args> {
-      public getTabletServerStatus() {
-        super("getTabletServerStatus");
-      }
-
-      @Override
-      public getTabletServerStatus_args getEmptyArgsInstance() {
-        return new getTabletServerStatus_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getTabletServerStatus_result getResult(I iface, getTabletServerStatus_args args) throws org.apache.thrift.TException {
-        getTabletServerStatus_result result = new getTabletServerStatus_result();
-        try {
-          result.success = iface.getTabletServerStatus(args.tinfo, args.credentials);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class getTabletStats<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTabletStats_args> {
-      public getTabletStats() {
-        super("getTabletStats");
-      }
-
-      @Override
-      public getTabletStats_args getEmptyArgsInstance() {
-        return new getTabletStats_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getTabletStats_result getResult(I iface, getTabletStats_args args) throws org.apache.thrift.TException {
-        getTabletStats_result result = new getTabletStats_result();
-        try {
-          result.success = iface.getTabletStats(args.tinfo, args.credentials, args.tableId);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class getHistoricalStats<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getHistoricalStats_args> {
-      public getHistoricalStats() {
-        super("getHistoricalStats");
-      }
-
-      @Override
-      public getHistoricalStats_args getEmptyArgsInstance() {
-        return new getHistoricalStats_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getHistoricalStats_result getResult(I iface, getHistoricalStats_args args) throws org.apache.thrift.TException {
-        getHistoricalStats_result result = new getHistoricalStats_result();
-        try {
-          result.success = iface.getHistoricalStats(args.tinfo, args.credentials);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class halt<I extends Iface> extends org.apache.thrift.ProcessFunction<I, halt_args> {
-      public halt() {
-        super("halt");
-      }
-
-      @Override
-      public halt_args getEmptyArgsInstance() {
-        return new halt_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public halt_result getResult(I iface, halt_args args) throws org.apache.thrift.TException {
-        halt_result result = new halt_result();
-        try {
-          iface.halt(args.tinfo, args.credentials, args.lock);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class fastHalt<I extends Iface> extends org.apache.thrift.ProcessFunction<I, fastHalt_args> {
-      public fastHalt() {
-        super("fastHalt");
-      }
-
-      @Override
-      public fastHalt_args getEmptyArgsInstance() {
-        return new fastHalt_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, fastHalt_args args) throws org.apache.thrift.TException {
-        iface.fastHalt(args.tinfo, args.credentials, args.lock);
-        return null;
-      }
-    }
-
-    public static class getActiveCompactions<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveCompactions_args> {
-      public getActiveCompactions() {
-        super("getActiveCompactions");
-      }
-
-      @Override
-      public getActiveCompactions_args getEmptyArgsInstance() {
-        return new getActiveCompactions_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getActiveCompactions_result getResult(I iface, getActiveCompactions_args args) throws org.apache.thrift.TException {
-        getActiveCompactions_result result = new getActiveCompactions_result();
-        try {
-          result.success = iface.getActiveCompactions(args.tinfo, args.credentials);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class removeLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, removeLogs_args> {
-      public removeLogs() {
-        super("removeLogs");
-      }
-
-      @Override
-      public removeLogs_args getEmptyArgsInstance() {
-        return new removeLogs_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, removeLogs_args args) throws org.apache.thrift.TException {
-        iface.removeLogs(args.tinfo, args.credentials, args.filenames);
-        return null;
-      }
-    }
-
-    public static class getActiveLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveLogs_args> {
-      public getActiveLogs() {
-        super("getActiveLogs");
-      }
-
-      @Override
-      public getActiveLogs_args getEmptyArgsInstance() {
-        return new getActiveLogs_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getActiveLogs_result getResult(I iface, getActiveLogs_args args) throws org.apache.thrift.TException {
-        getActiveLogs_result result = new getActiveLogs_result();
-        result.success = iface.getActiveLogs(args.tinfo, args.credentials);
-        return result;
-      }
-    }
-
-    public static class startGetSummaries<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummaries_args> {
-      public startGetSummaries() {
-        super("startGetSummaries");
-      }
-
-      @Override
-      public startGetSummaries_args getEmptyArgsInstance() {
-        return new startGetSummaries_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public startGetSummaries_result getResult(I iface, startGetSummaries_args args) throws org.apache.thrift.TException {
-        startGetSummaries_result result = new startGetSummaries_result();
-        try {
-          result.success = iface.startGetSummaries(args.tinfo, args.credentials, args.request);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException tope) {
-          result.tope = tope;
-        }
-        return result;
-      }
-    }
-
-    public static class startGetSummariesForPartition<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummariesForPartition_args> {
-      public startGetSummariesForPartition() {
-        super("startGetSummariesForPartition");
-      }
-
-      @Override
-      public startGetSummariesForPartition_args getEmptyArgsInstance() {
-        return new startGetSummariesForPartition_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public startGetSummariesForPartition_result getResult(I iface, startGetSummariesForPartition_args args) throws org.apache.thrift.TException {
-        startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
-        try {
-          result.success = iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class startGetSummariesFromFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummariesFromFiles_args> {
-      public startGetSummariesFromFiles() {
-        super("startGetSummariesFromFiles");
-      }
-
-      @Override
-      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
-        return new startGetSummariesFromFiles_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public startGetSummariesFromFiles_result getResult(I iface, startGetSummariesFromFiles_args args) throws org.apache.thrift.TException {
-        startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
-        try {
-          result.success = iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class contiuneGetSummaries<I extends Iface> extends org.apache.thrift.ProcessFunction<I, contiuneGetSummaries_args> {
-      public contiuneGetSummaries() {
-        super("contiuneGetSummaries");
-      }
-
-      @Override
-      public contiuneGetSummaries_args getEmptyArgsInstance() {
-        return new contiuneGetSummaries_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public contiuneGetSummaries_result getResult(I iface, contiuneGetSummaries_args args) throws org.apache.thrift.TException {
-        contiuneGetSummaries_result result = new contiuneGetSummaries_result();
-        try {
-          result.success = iface.contiuneGetSummaries(args.tinfo, args.sessionId);
-        } catch (NoSuchScanIDException nssi) {
-          result.nssi = nssi;
-        }
-        return result;
-      }
-    }
-
-    public static class getCompactionQueueInfo<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionQueueInfo_args> {
-      public getCompactionQueueInfo() {
-        super("getCompactionQueueInfo");
-      }
-
-      @Override
-      public getCompactionQueueInfo_args getEmptyArgsInstance() {
-        return new getCompactionQueueInfo_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public getCompactionQueueInfo_result getResult(I iface, getCompactionQueueInfo_args args) throws org.apache.thrift.TException {
-        getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
-        try {
-          result.success = iface.getCompactionQueueInfo(args.tinfo, args.credentials);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class reserveCompactionJob<I extends Iface> extends org.apache.thrift.ProcessFunction<I, reserveCompactionJob_args> {
-      public reserveCompactionJob() {
-        super("reserveCompactionJob");
-      }
-
-      @Override
-      public reserveCompactionJob_args getEmptyArgsInstance() {
-        return new reserveCompactionJob_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return false;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public reserveCompactionJob_result getResult(I iface, reserveCompactionJob_args args) throws org.apache.thrift.TException {
-        reserveCompactionJob_result result = new reserveCompactionJob_result();
-        try {
-          result.success = iface.reserveCompactionJob(args.tinfo, args.credentials, args.queueName, args.priority, args.compactor, args.externalCompactionId);
-        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
-          result.sec = sec;
-        }
-        return result;
-      }
-    }
-
-    public static class compactionJobFinished<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionJobFinished_args> {
-      public compactionJobFinished() {
-        super("compactionJobFinished");
-      }
-
-      @Override
-      public compactionJobFinished_args getEmptyArgsInstance() {
-        return new compactionJobFinished_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, compactionJobFinished_args args) throws org.apache.thrift.TException {
-        iface.compactionJobFinished(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.fileSize, args.entries);
-        return null;
-      }
-    }
-
-    public static class compactionJobFailed<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionJobFailed_args> {
-      public compactionJobFailed() {
-        super("compactionJobFailed");
-      }
-
-      @Override
-      public compactionJobFailed_args getEmptyArgsInstance() {
-        return new compactionJobFailed_args();
-      }
-
-      @Override
-      protected boolean isOneway() {
-        return true;
-      }
-
-      @Override
-      protected boolean rethrowUnhandledExceptions() {
-        return false;
-      }
-
-      @Override
-      public org.apache.thrift.TBase getResult(I iface, compactionJobFailed_args args) throws org.apache.thrift.TException {
-        iface.compactionJobFailed(args.tinfo, args.credentials, args.externalCompactionId, 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("startUpdate", new startUpdate());
-      processMap.put("applyUpdates", new applyUpdates());
-      processMap.put("closeUpdate", new closeUpdate());
-      processMap.put("cancelUpdate", new cancelUpdate());
-      processMap.put("update", new update());
-      processMap.put("startConditionalUpdate", new startConditionalUpdate());
-      processMap.put("conditionalUpdate", new conditionalUpdate());
-      processMap.put("invalidateConditionalUpdate", new invalidateConditionalUpdate());
-      processMap.put("closeConditionalUpdate", new closeConditionalUpdate());
-      processMap.put("bulkImport", new bulkImport());
-      processMap.put("loadFiles", new loadFiles());
-      processMap.put("splitTablet", new splitTablet());
-      processMap.put("loadTablet", new loadTablet());
-      processMap.put("unloadTablet", new unloadTablet());
-      processMap.put("flush", new flush());
-      processMap.put("flushTablet", new flushTablet());
-      processMap.put("chop", new chop());
-      processMap.put("compact", new compact());
-      processMap.put("getTabletServerStatus", new getTabletServerStatus());
-      processMap.put("getTabletStats", new getTabletStats());
-      processMap.put("getHistoricalStats", new getHistoricalStats());
-      processMap.put("halt", new halt());
-      processMap.put("fastHalt", new fastHalt());
-      processMap.put("getActiveCompactions", new getActiveCompactions());
-      processMap.put("removeLogs", new removeLogs());
-      processMap.put("getActiveLogs", new getActiveLogs());
-      processMap.put("startGetSummaries", new startGetSummaries());
-      processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
-      processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
-      processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
-      processMap.put("getCompactionQueueInfo", new getCompactionQueueInfo());
-      processMap.put("reserveCompactionJob", new reserveCompactionJob());
-      processMap.put("compactionJobFinished", new compactionJobFinished());
-      processMap.put("compactionJobFailed", new compactionJobFailed());
-      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 NoSuchScanIDException) {
-              result.nssi = (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 update<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, update_args, Void> {
-      public update() {
-        super("update");
-      }
-
-      @Override
-      public update_args getEmptyArgsInstance() {
-        return new update_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) {
-            update_result result = new update_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;
-            update_result result = new update_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 NotServingTabletException) {
-              result.nste = (NotServingTabletException) e;
-              result.setNsteIsSet(true);
-              msg = result;
-            } else if (e instanceof ConstraintViolationException) {
-              result.cve = (ConstraintViolationException) e;
-              result.setCveIsSet(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, update_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.update(args.tinfo, args.credentials, args.keyExtent, args.mutation, args.durability,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 NoSuchScanIDException) {
-              result.nssi = (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 bulkImport<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, bulkImport_args, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> {
-      public bulkImport() {
-        super("bulkImport");
-      }
-
-      @Override
-      public bulkImport_args getEmptyArgsInstance() {
-        return new bulkImport_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> 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.TKeyExtent>>() { 
-          @Override
-          public void onComplete(java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> o) {
-            bulkImport_result result = new bulkImport_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;
-            bulkImport_result result = new bulkImport_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, bulkImport_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>> resultHandler) throws org.apache.thrift.TException {
-        iface.bulkImport(args.tinfo, args.credentials, args.tid, args.files, args.setTime,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);
-      }
-    }
-
-    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 NotServingTabletException) {
-              result.nste = (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 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 flush<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flush_args, Void> {
-      public flush() {
-        super("flush");
-      }
-
-      @Override
-      public flush_args getEmptyArgsInstance() {
-        return new flush_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, flush_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,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);
-      }
-    }
-
-    public static class chop<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, chop_args, Void> {
-      public chop() {
-        super("chop");
-      }
-
-      @Override
-      public chop_args getEmptyArgsInstance() {
-        return new chop_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, chop_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.chop(args.tinfo, args.credentials, args.lock, args.extent,resultHandler);
-      }
-    }
-
-    public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
-      public compact() {
-        super("compact");
-      }
-
-      @Override
-      public compact_args getEmptyArgsInstance() {
-        return new compact_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, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
-      }
-    }
-
-    public static class getTabletServerStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletServerStatus_args, org.apache.accumulo.core.master.thrift.TabletServerStatus> {
-      public getTabletServerStatus() {
-        super("getTabletServerStatus");
-      }
-
-      @Override
-      public getTabletServerStatus_args getEmptyArgsInstance() {
-        return new getTabletServerStatus_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> 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.master.thrift.TabletServerStatus>() { 
-          @Override
-          public void onComplete(org.apache.accumulo.core.master.thrift.TabletServerStatus o) {
-            getTabletServerStatus_result result = new getTabletServerStatus_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;
-            getTabletServerStatus_result result = new getTabletServerStatus_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, getTabletServerStatus_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.master.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException {
-        iface.getTabletServerStatus(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class getTabletStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletStats_args, java.util.List<TabletStats>> {
-      public getTabletStats() {
-        super("getTabletStats");
-      }
-
-      @Override
-      public getTabletStats_args getEmptyArgsInstance() {
-        return new getTabletStats_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> 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<TabletStats>>() { 
-          @Override
-          public void onComplete(java.util.List<TabletStats> o) {
-            getTabletStats_result result = new getTabletStats_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;
-            getTabletStats_result result = new getTabletStats_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, getTabletStats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException {
-        iface.getTabletStats(args.tinfo, args.credentials, args.tableId,resultHandler);
-      }
-    }
-
-    public static class getHistoricalStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getHistoricalStats_args, TabletStats> {
-      public getHistoricalStats() {
-        super("getHistoricalStats");
-      }
-
-      @Override
-      public getHistoricalStats_args getEmptyArgsInstance() {
-        return new getHistoricalStats_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<TabletStats> 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<TabletStats>() { 
-          @Override
-          public void onComplete(TabletStats o) {
-            getHistoricalStats_result result = new getHistoricalStats_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;
-            getHistoricalStats_result result = new getHistoricalStats_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, getHistoricalStats_args args, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException {
-        iface.getHistoricalStats(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class halt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, halt_args, Void> {
-      public halt() {
-        super("halt");
-      }
-
-      @Override
-      public halt_args getEmptyArgsInstance() {
-        return new halt_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) {
-            halt_result result = new halt_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;
-            halt_result result = new halt_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, halt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.halt(args.tinfo, args.credentials, args.lock,resultHandler);
-      }
-    }
-
-    public static class fastHalt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fastHalt_args, Void> {
-      public fastHalt() {
-        super("fastHalt");
-      }
-
-      @Override
-      public fastHalt_args getEmptyArgsInstance() {
-        return new fastHalt_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, fastHalt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.fastHalt(args.tinfo, args.credentials, args.lock,resultHandler);
-      }
-    }
-
-    public static class getActiveCompactions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveCompactions_args, java.util.List<ActiveCompaction>> {
-      public getActiveCompactions() {
-        super("getActiveCompactions");
-      }
-
-      @Override
-      public getActiveCompactions_args getEmptyArgsInstance() {
-        return new getActiveCompactions_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> 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<ActiveCompaction>>() { 
-          @Override
-          public void onComplete(java.util.List<ActiveCompaction> o) {
-            getActiveCompactions_result result = new getActiveCompactions_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;
-            getActiveCompactions_result result = new getActiveCompactions_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, getActiveCompactions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
-        iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class removeLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, removeLogs_args, Void> {
-      public removeLogs() {
-        super("removeLogs");
-      }
-
-      @Override
-      public removeLogs_args getEmptyArgsInstance() {
-        return new removeLogs_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, removeLogs_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.removeLogs(args.tinfo, args.credentials, args.filenames,resultHandler);
-      }
-    }
-
-    public static class getActiveLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveLogs_args, java.util.List<java.lang.String>> {
-      public getActiveLogs() {
-        super("getActiveLogs");
-      }
-
-      @Override
-      public getActiveLogs_args getEmptyArgsInstance() {
-        return new getActiveLogs_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) {
-            getActiveLogs_result result = new getActiveLogs_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;
-            getActiveLogs_result result = new getActiveLogs_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, getActiveLogs_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
-        iface.getActiveLogs(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class startGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public startGetSummaries() {
-        super("startGetSummaries");
-      }
-
-      @Override
-      public startGetSummaries_args getEmptyArgsInstance() {
-        return new startGetSummaries_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
-          @Override
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            startGetSummaries_result result = new startGetSummaries_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;
-            startGetSummaries_result result = new startGetSummaries_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.clientImpl.thrift.ThriftTableOperationException) {
-              result.tope = (org.apache.accumulo.core.clientImpl.thrift.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, startGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.startGetSummaries(args.tinfo, args.credentials, args.request,resultHandler);
-      }
-    }
-
-    public static class startGetSummariesForPartition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesForPartition_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public startGetSummariesForPartition() {
-        super("startGetSummariesForPartition");
-      }
-
-      @Override
-      public startGetSummariesForPartition_args getEmptyArgsInstance() {
-        return new startGetSummariesForPartition_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
-          @Override
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            startGetSummariesForPartition_result result = new startGetSummariesForPartition_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;
-            startGetSummariesForPartition_result result = new startGetSummariesForPartition_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, startGetSummariesForPartition_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder,resultHandler);
-      }
-    }
-
-    public static class startGetSummariesFromFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesFromFiles_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public startGetSummariesFromFiles() {
-        super("startGetSummariesFromFiles");
-      }
-
-      @Override
-      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
-        return new startGetSummariesFromFiles_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
-          @Override
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_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;
-            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_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, startGetSummariesFromFiles_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files,resultHandler);
-      }
-    }
-
-    public static class contiuneGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, contiuneGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
-      public contiuneGetSummaries() {
-        super("contiuneGetSummaries");
-      }
-
-      @Override
-      public contiuneGetSummaries_args getEmptyArgsInstance() {
-        return new contiuneGetSummaries_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
-          @Override
-          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
-            contiuneGetSummaries_result result = new contiuneGetSummaries_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;
-            contiuneGetSummaries_result result = new contiuneGetSummaries_result();
-            if (e instanceof NoSuchScanIDException) {
-              result.nssi = (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, contiuneGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
-        iface.contiuneGetSummaries(args.tinfo, args.sessionId,resultHandler);
-      }
-    }
-
-    public static class getCompactionQueueInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionQueueInfo_args, java.util.List<TCompactionQueueSummary>> {
-      public getCompactionQueueInfo() {
-        super("getCompactionQueueInfo");
-      }
-
-      @Override
-      public getCompactionQueueInfo_args getEmptyArgsInstance() {
-        return new getCompactionQueueInfo_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> 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<TCompactionQueueSummary>>() { 
-          @Override
-          public void onComplete(java.util.List<TCompactionQueueSummary> o) {
-            getCompactionQueueInfo_result result = new getCompactionQueueInfo_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;
-            getCompactionQueueInfo_result result = new getCompactionQueueInfo_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, getCompactionQueueInfo_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
-        iface.getCompactionQueueInfo(args.tinfo, args.credentials,resultHandler);
-      }
-    }
-
-    public static class reserveCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, reserveCompactionJob_args, TExternalCompactionJob> {
-      public reserveCompactionJob() {
-        super("reserveCompactionJob");
-      }
-
-      @Override
-      public reserveCompactionJob_args getEmptyArgsInstance() {
-        return new reserveCompactionJob_args();
-      }
-
-      @Override
-      public org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> 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<TExternalCompactionJob>() { 
-          @Override
-          public void onComplete(TExternalCompactionJob o) {
-            reserveCompactionJob_result result = new reserveCompactionJob_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;
-            reserveCompactionJob_result result = new reserveCompactionJob_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, reserveCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
-        iface.reserveCompactionJob(args.tinfo, args.credentials, args.queueName, args.priority, args.compactor, args.externalCompactionId,resultHandler);
-      }
-    }
-
-    public static class compactionJobFinished<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFinished_args, Void> {
-      public compactionJobFinished() {
-        super("compactionJobFinished");
-      }
-
-      @Override
-      public compactionJobFinished_args getEmptyArgsInstance() {
-        return new compactionJobFinished_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, compactionJobFinished_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compactionJobFinished(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.fileSize, args.entries,resultHandler);
-      }
-    }
-
-    public static class compactionJobFailed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFailed_args, Void> {
-      public compactionJobFailed() {
-        super("compactionJobFailed");
-      }
-
-      @Override
-      public compactionJobFailed_args getEmptyArgsInstance() {
-        return new compactionJobFailed_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, compactionJobFailed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
-        iface.compactionJobFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent,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.trace.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.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.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startUpdate_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 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.trace.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.trace.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.tabletserver.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.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.durability = org.apache.accumulo.core.tabletserver.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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public applyUpdates_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;
-      }
-    }
-
-    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.trace.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.trace.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 _list268 = iprot.readListBegin();
-                  struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list268.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem269;
-                  for (int _i270 = 0; _i270 < _list268.size; ++_i270)
-                  {
-                    _elem269 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
-                    _elem269.read(iprot);
-                    struct.mutations.add(_elem269);
-                  }
-                  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 _iter271 : struct.mutations)
-            {
-              _iter271.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 _iter272 : struct.mutations)
-            {
-              _iter272.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.trace.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 _list273 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.mutations = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TMutation>(_list273.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation _elem274;
-            for (int _i275 = 0; _i275 < _list273.size; ++_i275)
-            {
-              _elem274 = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
-              _elem274.read(iprot);
-              struct.mutations.add(_elem274);
-            }
-          }
-          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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public closeUpdate_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;
-      }
-    }
-
-    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.trace.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.trace.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.trace.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 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, 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,
-      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 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 NoSuchScanIDException getNssi() {
-      return this.nssi;
-    }
-
-    public closeUpdate_result setNssi(@org.apache.thrift.annotation.Nullable 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((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 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 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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public cancelUpdate_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;
-      }
-    }
-
-    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.trace.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.trace.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.trace.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 update_args implements org.apache.thrift.TBase<update_args, update_args._Fields>, java.io.Serializable, Cloneable, Comparable<update_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("update_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 KEY_EXTENT_FIELD_DESC = new org.apache.thrift.protocol.TField("keyExtent", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField MUTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("mutation", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new update_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new update_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation mutation; // 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)4, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      KEY_EXTENT((short)2, "keyExtent"),
-      MUTATION((short)3, "mutation"),
-      /**
-       * 
-       * @see TDurability
-       */
-      DURABILITY((short)5, "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 4: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 2: // KEY_EXTENT
-            return KEY_EXTENT;
-          case 3: // MUTATION
-            return MUTATION;
-          case 5: // 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.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.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.MUTATION, new org.apache.thrift.meta_data.FieldMetaData("mutation", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TMutation.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(update_args.class, metaDataMap);
-    }
-
-    public update_args() {
-    }
-
-    public update_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      org.apache.accumulo.core.dataImpl.thrift.TKeyExtent keyExtent,
-      org.apache.accumulo.core.dataImpl.thrift.TMutation mutation,
-      TDurability durability)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.keyExtent = keyExtent;
-      this.mutation = mutation;
-      this.durability = durability;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public update_args(update_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetKeyExtent()) {
-        this.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.keyExtent);
-      }
-      if (other.isSetMutation()) {
-        this.mutation = new org.apache.accumulo.core.dataImpl.thrift.TMutation(other.mutation);
-      }
-      if (other.isSetDurability()) {
-        this.durability = other.durability;
-      }
-    }
-
-    @Override
-    public update_args deepCopy() {
-      return new update_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.keyExtent = null;
-      this.mutation = null;
-      this.durability = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public update_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 update_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 getKeyExtent() {
-      return this.keyExtent;
-    }
-
-    public update_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;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TMutation getMutation() {
-      return this.mutation;
-    }
-
-    public update_args setMutation(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TMutation mutation) {
-      this.mutation = mutation;
-      return this;
-    }
-
-    public void unsetMutation() {
-      this.mutation = null;
-    }
-
-    /** Returns true if field mutation is set (has been assigned a value) and false otherwise */
-    public boolean isSetMutation() {
-      return this.mutation != null;
-    }
-
-    public void setMutationIsSet(boolean value) {
-      if (!value) {
-        this.mutation = null;
-      }
-    }
-
-    /**
-     * 
-     * @see TDurability
-     */
-    @org.apache.thrift.annotation.Nullable
-    public TDurability getDurability() {
-      return this.durability;
-    }
-
-    /**
-     * 
-     * @see TDurability
-     */
-    public update_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.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case KEY_EXTENT:
-        if (value == null) {
-          unsetKeyExtent();
-        } else {
-          setKeyExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
-        }
-        break;
-
-      case MUTATION:
-        if (value == null) {
-          unsetMutation();
-        } else {
-          setMutation((org.apache.accumulo.core.dataImpl.thrift.TMutation)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 KEY_EXTENT:
-        return getKeyExtent();
-
-      case MUTATION:
-        return getMutation();
-
-      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 KEY_EXTENT:
-        return isSetKeyExtent();
-      case MUTATION:
-        return isSetMutation();
-      case DURABILITY:
-        return isSetDurability();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof update_args)
-        return this.equals((update_args)that);
-      return false;
-    }
-
-    public boolean equals(update_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_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_mutation = true && this.isSetMutation();
-      boolean that_present_mutation = true && that.isSetMutation();
-      if (this_present_mutation || that_present_mutation) {
-        if (!(this_present_mutation && that_present_mutation))
-          return false;
-        if (!this.mutation.equals(that.mutation))
-          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 + ((isSetKeyExtent()) ? 131071 : 524287);
-      if (isSetKeyExtent())
-        hashCode = hashCode * 8191 + keyExtent.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetMutation()) ? 131071 : 524287);
-      if (isSetMutation())
-        hashCode = hashCode * 8191 + mutation.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetDurability()) ? 131071 : 524287);
-      if (isSetDurability())
-        hashCode = hashCode * 8191 + durability.getValue();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(update_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(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(isSetMutation(), other.isSetMutation());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetMutation()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutation, other.mutation);
-        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("update_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("keyExtent:");
-      if (this.keyExtent == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.keyExtent);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("mutation:");
-      if (this.mutation == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.mutation);
-      }
-      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();
-      }
-      if (keyExtent != null) {
-        keyExtent.validate();
-      }
-      if (mutation != null) {
-        mutation.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 update_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public update_argsStandardScheme getScheme() {
-        return new update_argsStandardScheme();
-      }
-    }
-
-    private static class update_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<update_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, update_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.trace.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: // 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 3: // MUTATION
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.mutation = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
-                struct.mutation.read(iprot);
-                struct.setMutationIsSet(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.tabletserver.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, update_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.keyExtent != null) {
-          oprot.writeFieldBegin(KEY_EXTENT_FIELD_DESC);
-          struct.keyExtent.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.mutation != null) {
-          oprot.writeFieldBegin(MUTATION_FIELD_DESC);
-          struct.mutation.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 update_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public update_argsTupleScheme getScheme() {
-        return new update_argsTupleScheme();
-      }
-    }
-
-    private static class update_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<update_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, update_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.isSetKeyExtent()) {
-          optionals.set(2);
-        }
-        if (struct.isSetMutation()) {
-          optionals.set(3);
-        }
-        if (struct.isSetDurability()) {
-          optionals.set(4);
-        }
-        oprot.writeBitSet(optionals, 5);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetKeyExtent()) {
-          struct.keyExtent.write(oprot);
-        }
-        if (struct.isSetMutation()) {
-          struct.mutation.write(oprot);
-        }
-        if (struct.isSetDurability()) {
-          oprot.writeI32(struct.durability.getValue());
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, update_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(5);
-        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.keyExtent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-          struct.keyExtent.read(iprot);
-          struct.setKeyExtentIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.mutation = new org.apache.accumulo.core.dataImpl.thrift.TMutation();
-          struct.mutation.read(iprot);
-          struct.setMutationIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.durability = org.apache.accumulo.core.tabletserver.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 update_result implements org.apache.thrift.TBase<update_result, update_result._Fields>, java.io.Serializable, Cloneable, Comparable<update_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("update_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.protocol.TField CVE_FIELD_DESC = new org.apache.thrift.protocol.TField("cve", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new update_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new update_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec; // required
-    public @org.apache.thrift.annotation.Nullable NotServingTabletException nste; // required
-    public @org.apache.thrift.annotation.Nullable ConstraintViolationException cve; // 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"),
-      CVE((short)3, "cve");
-
-      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;
-          case 3: // CVE
-            return CVE;
-          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, NotServingTabletException.class)));
-      tmpMap.put(_Fields.CVE, new org.apache.thrift.meta_data.FieldMetaData("cve", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ConstraintViolationException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(update_result.class, metaDataMap);
-    }
-
-    public update_result() {
-    }
-
-    public update_result(
-      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
-      NotServingTabletException nste,
-      ConstraintViolationException cve)
-    {
-      this();
-      this.sec = sec;
-      this.nste = nste;
-      this.cve = cve;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public update_result(update_result other) {
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-      if (other.isSetNste()) {
-        this.nste = new NotServingTabletException(other.nste);
-      }
-      if (other.isSetCve()) {
-        this.cve = new ConstraintViolationException(other.cve);
-      }
-    }
-
-    @Override
-    public update_result deepCopy() {
-      return new update_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.sec = null;
-      this.nste = null;
-      this.cve = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
-      return this.sec;
-    }
-
-    public update_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 NotServingTabletException getNste() {
-      return this.nste;
-    }
-
-    public update_result setNste(@org.apache.thrift.annotation.Nullable 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;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public ConstraintViolationException getCve() {
-      return this.cve;
-    }
-
-    public update_result setCve(@org.apache.thrift.annotation.Nullable ConstraintViolationException cve) {
-      this.cve = cve;
-      return this;
-    }
-
-    public void unsetCve() {
-      this.cve = null;
-    }
-
-    /** Returns true if field cve is set (has been assigned a value) and false otherwise */
-    public boolean isSetCve() {
-      return this.cve != null;
-    }
-
-    public void setCveIsSet(boolean value) {
-      if (!value) {
-        this.cve = 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((NotServingTabletException)value);
-        }
-        break;
-
-      case CVE:
-        if (value == null) {
-          unsetCve();
-        } else {
-          setCve((ConstraintViolationException)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();
-
-      case CVE:
-        return getCve();
-
-      }
-      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();
-      case CVE:
-        return isSetCve();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof update_result)
-        return this.equals((update_result)that);
-      return false;
-    }
-
-    public boolean equals(update_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;
-      }
-
-      boolean this_present_cve = true && this.isSetCve();
-      boolean that_present_cve = true && that.isSetCve();
-      if (this_present_cve || that_present_cve) {
-        if (!(this_present_cve && that_present_cve))
-          return false;
-        if (!this.cve.equals(that.cve))
-          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();
-
-      hashCode = hashCode * 8191 + ((isSetCve()) ? 131071 : 524287);
-      if (isSetCve())
-        hashCode = hashCode * 8191 + cve.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(update_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;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCve(), other.isSetCve());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCve()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cve, other.cve);
-        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("update_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;
-      if (!first) sb.append(", ");
-      sb.append("cve:");
-      if (this.cve == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.cve);
-      }
-      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 update_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public update_resultStandardScheme getScheme() {
-        return new update_resultStandardScheme();
-      }
-    }
-
-    private static class update_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<update_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, update_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 NotServingTabletException();
-                struct.nste.read(iprot);
-                struct.setNsteIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // CVE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.cve = new ConstraintViolationException();
-                struct.cve.read(iprot);
-                struct.setCveIsSet(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, update_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();
-        }
-        if (struct.cve != null) {
-          oprot.writeFieldBegin(CVE_FIELD_DESC);
-          struct.cve.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class update_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public update_resultTupleScheme getScheme() {
-        return new update_resultTupleScheme();
-      }
-    }
-
-    private static class update_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<update_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, update_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);
-        }
-        if (struct.isSetCve()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-        if (struct.isSetNste()) {
-          struct.nste.write(oprot);
-        }
-        if (struct.isSetCve()) {
-          struct.cve.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, update_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)) {
-          struct.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException();
-          struct.sec.read(iprot);
-          struct.setSecIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.nste = new NotServingTabletException();
-          struct.nste.read(iprot);
-          struct.setNsteIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.cve = new ConstraintViolationException();
-          struct.cve.read(iprot);
-          struct.setCveIsSet(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.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.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.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.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startConditionalUpdate_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 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.trace.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.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // AUTHORIZATIONS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list276 = iprot.readListBegin();
-                  struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list276.size);
-                  @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem277;
-                  for (int _i278 = 0; _i278 < _list276.size; ++_i278)
-                  {
-                    _elem277 = iprot.readBinary();
-                    struct.authorizations.add(_elem277);
-                  }
-                  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.tabletserver.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 _iter279 : struct.authorizations)
-            {
-              oprot.writeBinary(_iter279);
-            }
-            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 _iter280 : struct.authorizations)
-            {
-              oprot.writeBinary(_iter280);
-            }
-          }
-        }
-        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.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)) {
-          {
-            org.apache.thrift.protocol.TList _list281 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.authorizations = new java.util.ArrayList<java.nio.ByteBuffer>(_list281.size);
-            @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer _elem282;
-            for (int _i283 = 0; _i283 < _list281.size; ++_i283)
-            {
-              _elem282 = iprot.readBinary();
-              struct.authorizations.add(_elem282);
-            }
-          }
-          struct.setAuthorizationsIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.tableID = iprot.readString();
-          struct.setTableIDIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.durability = org.apache.accumulo.core.tabletserver.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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public conditionalUpdate_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;
-      }
-    }
-
-    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.trace.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.trace.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 _map284 = 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*_map284.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key285;
-                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val286;
-                  for (int _i287 = 0; _i287 < _map284.size; ++_i287)
-                  {
-                    _key285 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key285.read(iprot);
-                    {
-                      org.apache.thrift.protocol.TList _list288 = iprot.readListBegin();
-                      _val286 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list288.size);
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem289;
-                      for (int _i290 = 0; _i290 < _list288.size; ++_i290)
-                      {
-                        _elem289 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
-                        _elem289.read(iprot);
-                        _val286.add(_elem289);
-                      }
-                      iprot.readListEnd();
-                    }
-                    struct.mutations.put(_key285, _val286);
-                  }
-                  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 _list291 = iprot.readListBegin();
-                  struct.symbols = new java.util.ArrayList<java.lang.String>(_list291.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem292;
-                  for (int _i293 = 0; _i293 < _list291.size; ++_i293)
-                  {
-                    _elem292 = iprot.readString();
-                    struct.symbols.add(_elem292);
-                  }
-                  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>> _iter294 : struct.mutations.entrySet())
-            {
-              _iter294.getKey().write(oprot);
-              {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter294.getValue().size()));
-                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter295 : _iter294.getValue())
-                {
-                  _iter295.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 _iter296 : struct.symbols)
-            {
-              oprot.writeString(_iter296);
-            }
-            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>> _iter297 : struct.mutations.entrySet())
-            {
-              _iter297.getKey().write(oprot);
-              {
-                oprot.writeI32(_iter297.getValue().size());
-                for (org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _iter298 : _iter297.getValue())
-                {
-                  _iter298.write(oprot);
-                }
-              }
-            }
-          }
-        }
-        if (struct.isSetSymbols()) {
-          {
-            oprot.writeI32(struct.symbols.size());
-            for (java.lang.String _iter299 : struct.symbols)
-            {
-              oprot.writeString(_iter299);
-            }
-          }
-        }
-      }
-
-      @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.trace.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 _map300 = 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*_map300.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key301;
-            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation> _val302;
-            for (int _i303 = 0; _i303 < _map300.size; ++_i303)
-            {
-              _key301 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key301.read(iprot);
-              {
-                org.apache.thrift.protocol.TList _list304 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-                _val302 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation>(_list304.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation _elem305;
-                for (int _i306 = 0; _i306 < _list304.size; ++_i306)
-                {
-                  _elem305 = new org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation();
-                  _elem305.read(iprot);
-                  _val302.add(_elem305);
-                }
-              }
-              struct.mutations.put(_key301, _val302);
-            }
-          }
-          struct.setMutationsIsSet(true);
-        }
-        if (incoming.get(3)) {
-          {
-            org.apache.thrift.protocol.TList _list307 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.symbols = new java.util.ArrayList<java.lang.String>(_list307.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem308;
-            for (int _i309 = 0; _i309 < _list307.size; ++_i309)
-            {
-              _elem308 = iprot.readString();
-              struct.symbols.add(_elem308);
-            }
-          }
-          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 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, 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,
-      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 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 NoSuchScanIDException getNssi() {
-      return this.nssi;
-    }
-
-    public conditionalUpdate_result setNssi(@org.apache.thrift.annotation.Nullable 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((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 _list310 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list310.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem311;
-                  for (int _i312 = 0; _i312 < _list310.size; ++_i312)
-                  {
-                    _elem311 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
-                    _elem311.read(iprot);
-                    struct.success.add(_elem311);
-                  }
-                  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 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 _iter313 : struct.success)
-            {
-              _iter313.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 _iter314 : struct.success)
-            {
-              _iter314.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 _list315 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TCMResult>(_list315.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TCMResult _elem316;
-            for (int _i317 = 0; _i317 < _list315.size; ++_i317)
-            {
-              _elem316 = new org.apache.accumulo.core.dataImpl.thrift.TCMResult();
-              _elem316.read(iprot);
-              struct.success.add(_elem316);
-            }
-          }
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.nssi = new 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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public invalidateConditionalUpdate_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;
-      }
-    }
-
-    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.trace.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.trace.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.trace.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.trace.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.trace.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.trace.thrift.TInfo tinfo,
-      long sessID)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.sessID = sessID;
-      setSessIDIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public closeConditionalUpdate_args(closeConditionalUpdate_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      this.sessID = other.sessID;
-    }
-
-    @Override
-    public closeConditionalUpdate_args deepCopy() {
-      return new closeConditionalUpdate_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      setSessIDIsSet(false);
-      this.sessID = 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public closeConditionalUpdate_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;
-      }
-    }
-
-    public long getSessID() {
-      return this.sessID;
-    }
-
-    public closeConditionalUpdate_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.trace.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 closeConditionalUpdate_args)
-        return this.equals((closeConditionalUpdate_args)that);
-      return false;
-    }
-
-    public boolean equals(closeConditionalUpdate_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(closeConditionalUpdate_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("closeConditionalUpdate_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 closeConditionalUpdate_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public closeConditionalUpdate_argsStandardScheme getScheme() {
-        return new closeConditionalUpdate_argsStandardScheme();
-      }
-    }
-
-    private static class closeConditionalUpdate_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<closeConditionalUpdate_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, closeConditionalUpdate_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // 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, closeConditionalUpdate_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 closeConditionalUpdate_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public closeConditionalUpdate_argsTupleScheme getScheme() {
-        return new closeConditionalUpdate_argsTupleScheme();
-      }
-    }
-
-    private static class closeConditionalUpdate_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<closeConditionalUpdate_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, closeConditionalUpdate_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, closeConditionalUpdate_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.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 bulkImport_args implements org.apache.thrift.TBase<bulkImport_args, bulkImport_args._Fields>, java.io.Serializable, Cloneable, Comparable<bulkImport_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("bulkImport_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)3);
-    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 TID_FIELD_DESC = new org.apache.thrift.protocol.TField("tid", org.apache.thrift.protocol.TType.I64, (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.MAP, (short)2);
-    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)5);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new bulkImport_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new bulkImport_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.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files; // 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)3, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      TID((short)4, "tid"),
-      FILES((short)2, "files"),
-      SET_TIME((short)5, "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 3: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 4: // TID
-            return TID;
-          case 2: // FILES
-            return FILES;
-          case 5: // 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.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.MAP          , "TabletFiles")));
-      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(bulkImport_args.class, metaDataMap);
-    }
-
-    public bulkImport_args() {
-    }
-
-    public bulkImport_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      long tid,
-      java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files,
-      boolean setTime)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.tid = tid;
-      setTidIsSet(true);
-      this.files = files;
-      this.setTime = setTime;
-      setSetTimeIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public bulkImport_args(bulkImport_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.isSetFiles()) {
-        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> __this__files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(other.files.size());
-        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> other_element : other.files.entrySet()) {
-
-          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
-          java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> other_element_value = other_element.getValue();
-
-          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__files_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
-
-          java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> __this__files_copy_value = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(other_element_value.size());
-          for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> other_element_value_element : other_element_value.entrySet()) {
-
-            java.lang.String other_element_value_element_key = other_element_value_element.getKey();
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo other_element_value_element_value = other_element_value_element.getValue();
-
-            java.lang.String __this__files_copy_value_copy_key = other_element_value_element_key;
-
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo __this__files_copy_value_copy_value = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo(other_element_value_element_value);
-
-            __this__files_copy_value.put(__this__files_copy_value_copy_key, __this__files_copy_value_copy_value);
-          }
-
-          __this__files.put(__this__files_copy_key, __this__files_copy_value);
-        }
-        this.files = __this__files;
-      }
-      this.setTime = other.setTime;
-    }
-
-    @Override
-    public bulkImport_args deepCopy() {
-      return new bulkImport_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      setTidIsSet(false);
-      this.tid = 0;
-      this.files = null;
-      setSetTimeIsSet(false);
-      this.setTime = false;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public bulkImport_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 bulkImport_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 bulkImport_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);
-    }
-
-    public int getFilesSize() {
-      return (this.files == null) ? 0 : this.files.size();
-    }
-
-    public void putToFiles(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> val) {
-      if (this.files == null) {
-        this.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>();
-      }
-      this.files.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> getFiles() {
-      return this.files;
-    }
-
-    public bulkImport_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> 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;
-      }
-    }
-
-    public boolean isSetTime() {
-      return this.setTime;
-    }
-
-    public bulkImport_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 FILES:
-        if (value == null) {
-          unsetFiles();
-        } else {
-          setFiles((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>)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 FILES:
-        return getFiles();
-
-      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 FILES:
-        return isSetFiles();
-      case SET_TIME:
-        return isSetSetTime();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof bulkImport_args)
-        return this.equals((bulkImport_args)that);
-      return false;
-    }
-
-    public boolean equals(bulkImport_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_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_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 + ((isSetFiles()) ? 131071 : 524287);
-      if (isSetFiles())
-        hashCode = hashCode * 8191 + files.hashCode();
-
-      hashCode = hashCode * 8191 + ((setTime) ? 131071 : 524287);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(bulkImport_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(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(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("bulkImport_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("files:");
-      if (this.files == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.files);
-      }
-      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 bulkImport_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImport_argsStandardScheme getScheme() {
-        return new bulkImport_argsStandardScheme();
-      }
-    }
-
-    private static class bulkImport_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<bulkImport_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImport_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 3: // 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 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: // 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 2: // FILES
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map318 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map318.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key319;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val320;
-                  for (int _i321 = 0; _i321 < _map318.size; ++_i321)
-                  {
-                    _key319 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key319.read(iprot);
-                    {
-                      org.apache.thrift.protocol.TMap _map322 = iprot.readMapBegin();
-                      _val320 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map322.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key323;
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val324;
-                      for (int _i325 = 0; _i325 < _map322.size; ++_i325)
-                      {
-                        _key323 = iprot.readString();
-                        _val324 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
-                        _val324.read(iprot);
-                        _val320.put(_key323, _val324);
-                      }
-                      iprot.readMapEnd();
-                    }
-                    struct.files.put(_key319, _val320);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setFilesIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // 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, bulkImport_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.files != null) {
-          oprot.writeFieldBegin(FILES_FIELD_DESC);
-          {
-            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, struct.files.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter326 : struct.files.entrySet())
-            {
-              _iter326.getKey().write(oprot);
-              {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter326.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter327 : _iter326.getValue().entrySet())
-                {
-                  oprot.writeString(_iter327.getKey());
-                  _iter327.getValue().write(oprot);
-                }
-                oprot.writeMapEnd();
-              }
-            }
-            oprot.writeMapEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        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();
-        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
-        oprot.writeBool(struct.setTime);
-        oprot.writeFieldEnd();
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class bulkImport_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImport_argsTupleScheme getScheme() {
-        return new bulkImport_argsTupleScheme();
-      }
-    }
-
-    private static class bulkImport_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<bulkImport_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, bulkImport_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.isSetFiles()) {
-          optionals.set(3);
-        }
-        if (struct.isSetSetTime()) {
-          optionals.set(4);
-        }
-        oprot.writeBitSet(optionals, 5);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetTid()) {
-          oprot.writeI64(struct.tid);
-        }
-        if (struct.isSetFiles()) {
-          {
-            oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter328 : struct.files.entrySet())
-            {
-              _iter328.getKey().write(oprot);
-              {
-                oprot.writeI32(_iter328.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter329 : _iter328.getValue().entrySet())
-                {
-                  oprot.writeString(_iter329.getKey());
-                  _iter329.getValue().write(oprot);
-                }
-              }
-            }
-          }
-        }
-        if (struct.isSetSetTime()) {
-          oprot.writeBool(struct.setTime);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, bulkImport_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(5);
-        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)) {
-          {
-            org.apache.thrift.protocol.TMap _map330 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP); 
-            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map330.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key331;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val332;
-            for (int _i333 = 0; _i333 < _map330.size; ++_i333)
-            {
-              _key331 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key331.read(iprot);
-              {
-                org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
-                _val332 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map334.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key335;
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val336;
-                for (int _i337 = 0; _i337 < _map334.size; ++_i337)
-                {
-                  _key335 = iprot.readString();
-                  _val336 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
-                  _val336.read(iprot);
-                  _val332.put(_key335, _val336);
-                }
-              }
-              struct.files.put(_key331, _val332);
-            }
-          }
-          struct.setFilesIsSet(true);
-        }
-        if (incoming.get(4)) {
-          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 bulkImport_result implements org.apache.thrift.TBase<bulkImport_result, bulkImport_result._Fields>, java.io.Serializable, Cloneable, Comparable<bulkImport_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("bulkImport_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new bulkImport_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new bulkImport_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> 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.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.TKeyExtent.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(bulkImport_result.class, metaDataMap);
-    }
-
-    public bulkImport_result() {
-    }
-
-    public bulkImport_result(
-      java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> 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 bulkImport_result(bulkImport_result other) {
-      if (other.isSetSuccess()) {
-        java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> __this__success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>(other.success.size());
-        for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element : other.success) {
-          __this__success.add(new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element));
-        }
-        this.success = __this__success;
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public bulkImport_result deepCopy() {
-      return new bulkImport_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = 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.TKeyExtent> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>();
-      }
-      this.success.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> getSuccess() {
-      return this.success;
-    }
-
-    public bulkImport_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent> 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 bulkImport_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.util.List<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>)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 bulkImport_result)
-        return this.equals((bulkImport_result)that);
-      return false;
-    }
-
-    public boolean equals(bulkImport_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(bulkImport_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("bulkImport_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
-    }
-
-    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 bulkImport_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImport_resultStandardScheme getScheme() {
-        return new bulkImport_resultStandardScheme();
-      }
-    }
-
-    private static class bulkImport_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<bulkImport_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, bulkImport_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 _list338 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>(_list338.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem339;
-                  for (int _i340 = 0; _i340 < _list338.size; ++_i340)
-                  {
-                    _elem339 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _elem339.read(iprot);
-                    struct.success.add(_elem339);
-                  }
-                  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 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, bulkImport_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.TKeyExtent _iter341 : struct.success)
-            {
-              _iter341.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class bulkImport_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public bulkImport_resultTupleScheme getScheme() {
-        return new bulkImport_resultTupleScheme();
-      }
-    }
-
-    private static class bulkImport_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<bulkImport_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, bulkImport_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.writeI32(struct.success.size());
-            for (org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _iter342 : struct.success)
-            {
-              _iter342.write(oprot);
-            }
-          }
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, bulkImport_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 _list343 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent>(_list343.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _elem344;
-            for (int _i345 = 0; _i345 < _list343.size; ++_i345)
-            {
-              _elem344 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _elem344.read(iprot);
-              struct.success.add(_elem344);
-            }
-          }
-          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 loadFiles_args implements org.apache.thrift.TBase<loadFiles_args, loadFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<loadFiles_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("loadFiles_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 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 DIR_FIELD_DESC = new org.apache.thrift.protocol.TField("dir", 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.MAP, (short)5);
-    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)6);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new loadFiles_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new loadFiles_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 dir; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files; // 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)2, "credentials"),
-      TID((short)3, "tid"),
-      DIR((short)4, "dir"),
-      FILES((short)5, "files"),
-      SET_TIME((short)6, "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 2: // CREDENTIALS
-            return CREDENTIALS;
-          case 3: // TID
-            return TID;
-          case 4: // DIR
-            return DIR;
-          case 5: // FILES
-            return FILES;
-          case 6: // 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.DIR, new org.apache.thrift.meta_data.FieldMetaData("dir", 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.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
-              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TKeyExtent.class), 
-              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.dataImpl.thrift.MapFileInfo.class)))));
-      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(loadFiles_args.class, metaDataMap);
-    }
-
-    public loadFiles_args() {
-    }
-
-    public loadFiles_args(
-      org.apache.accumulo.core.trace.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,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files,
-      boolean setTime)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.tid = tid;
-      setTidIsSet(true);
-      this.dir = dir;
-      this.files = files;
-      this.setTime = setTime;
-      setSetTimeIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public loadFiles_args(loadFiles_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.isSetDir()) {
-        this.dir = other.dir;
-      }
-      if (other.isSetFiles()) {
-        java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> __this__files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(other.files.size());
-        for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> other_element : other.files.entrySet()) {
-
-          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent other_element_key = other_element.getKey();
-          java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> other_element_value = other_element.getValue();
-
-          org.apache.accumulo.core.dataImpl.thrift.TKeyExtent __this__files_copy_key = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other_element_key);
-
-          java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> __this__files_copy_value = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(other_element_value.size());
-          for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> other_element_value_element : other_element_value.entrySet()) {
-
-            java.lang.String other_element_value_element_key = other_element_value_element.getKey();
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo other_element_value_element_value = other_element_value_element.getValue();
-
-            java.lang.String __this__files_copy_value_copy_key = other_element_value_element_key;
-
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo __this__files_copy_value_copy_value = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo(other_element_value_element_value);
-
-            __this__files_copy_value.put(__this__files_copy_value_copy_key, __this__files_copy_value_copy_value);
-          }
-
-          __this__files.put(__this__files_copy_key, __this__files_copy_value);
-        }
-        this.files = __this__files;
-      }
-      this.setTime = other.setTime;
-    }
-
-    @Override
-    public loadFiles_args deepCopy() {
-      return new loadFiles_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      setTidIsSet(false);
-      this.tid = 0;
-      this.dir = null;
-      this.files = null;
-      setSetTimeIsSet(false);
-      this.setTime = false;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public loadFiles_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 loadFiles_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 loadFiles_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 getDir() {
-      return this.dir;
-    }
-
-    public loadFiles_args setDir(@org.apache.thrift.annotation.Nullable java.lang.String dir) {
-      this.dir = dir;
-      return this;
-    }
-
-    public void unsetDir() {
-      this.dir = null;
-    }
-
-    /** Returns true if field dir is set (has been assigned a value) and false otherwise */
-    public boolean isSetDir() {
-      return this.dir != null;
-    }
-
-    public void setDirIsSet(boolean value) {
-      if (!value) {
-        this.dir = null;
-      }
-    }
-
-    public int getFilesSize() {
-      return (this.files == null) ? 0 : this.files.size();
-    }
-
-    public void putToFiles(org.apache.accumulo.core.dataImpl.thrift.TKeyExtent key, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> val) {
-      if (this.files == null) {
-        this.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>();
-      }
-      this.files.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> getFiles() {
-      return this.files;
-    }
-
-    public loadFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> 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;
-      }
-    }
-
-    public boolean isSetTime() {
-      return this.setTime;
-    }
-
-    public loadFiles_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 DIR:
-        if (value == null) {
-          unsetDir();
-        } else {
-          setDir((java.lang.String)value);
-        }
-        break;
-
-      case FILES:
-        if (value == null) {
-          unsetFiles();
-        } else {
-          setFiles((java.util.Map<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>)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 DIR:
-        return getDir();
-
-      case FILES:
-        return getFiles();
-
-      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 DIR:
-        return isSetDir();
-      case FILES:
-        return isSetFiles();
-      case SET_TIME:
-        return isSetSetTime();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof loadFiles_args)
-        return this.equals((loadFiles_args)that);
-      return false;
-    }
-
-    public boolean equals(loadFiles_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_dir = true && this.isSetDir();
-      boolean that_present_dir = true && that.isSetDir();
-      if (this_present_dir || that_present_dir) {
-        if (!(this_present_dir && that_present_dir))
-          return false;
-        if (!this.dir.equals(that.dir))
-          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_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 + ((isSetDir()) ? 131071 : 524287);
-      if (isSetDir())
-        hashCode = hashCode * 8191 + dir.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
-      if (isSetFiles())
-        hashCode = hashCode * 8191 + files.hashCode();
-
-      hashCode = hashCode * 8191 + ((setTime) ? 131071 : 524287);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(loadFiles_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(isSetDir(), other.isSetDir());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetDir()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dir, other.dir);
-        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(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("loadFiles_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("dir:");
-      if (this.dir == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.dir);
-      }
-      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("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 loadFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public loadFiles_argsStandardScheme getScheme() {
-        return new loadFiles_argsStandardScheme();
-      }
-    }
-
-    private static class loadFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<loadFiles_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, loadFiles_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            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: // DIR
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.dir = iprot.readString();
-                struct.setDirIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // FILES
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map346 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map346.size);
-                  @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key347;
-                  @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val348;
-                  for (int _i349 = 0; _i349 < _map346.size; ++_i349)
-                  {
-                    _key347 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-                    _key347.read(iprot);
-                    {
-                      org.apache.thrift.protocol.TMap _map350 = iprot.readMapBegin();
-                      _val348 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map350.size);
-                      @org.apache.thrift.annotation.Nullable java.lang.String _key351;
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val352;
-                      for (int _i353 = 0; _i353 < _map350.size; ++_i353)
-                      {
-                        _key351 = iprot.readString();
-                        _val352 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
-                        _val352.read(iprot);
-                        _val348.put(_key351, _val352);
-                      }
-                      iprot.readMapEnd();
-                    }
-                    struct.files.put(_key347, _val348);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setFilesIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // 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, loadFiles_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(TID_FIELD_DESC);
-        oprot.writeI64(struct.tid);
-        oprot.writeFieldEnd();
-        if (struct.dir != null) {
-          oprot.writeFieldBegin(DIR_FIELD_DESC);
-          oprot.writeString(struct.dir);
-          oprot.writeFieldEnd();
-        }
-        if (struct.files != null) {
-          oprot.writeFieldBegin(FILES_FIELD_DESC);
-          {
-            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP, struct.files.size()));
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter354 : struct.files.entrySet())
-            {
-              _iter354.getKey().write(oprot);
-              {
-                oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter354.getValue().size()));
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter355 : _iter354.getValue().entrySet())
-                {
-                  oprot.writeString(_iter355.getKey());
-                  _iter355.getValue().write(oprot);
-                }
-                oprot.writeMapEnd();
-              }
-            }
-            oprot.writeMapEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(SET_TIME_FIELD_DESC);
-        oprot.writeBool(struct.setTime);
-        oprot.writeFieldEnd();
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class loadFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public loadFiles_argsTupleScheme getScheme() {
-        return new loadFiles_argsTupleScheme();
-      }
-    }
-
-    private static class loadFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<loadFiles_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, loadFiles_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.isSetDir()) {
-          optionals.set(3);
-        }
-        if (struct.isSetFiles()) {
-          optionals.set(4);
-        }
-        if (struct.isSetSetTime()) {
-          optionals.set(5);
-        }
-        oprot.writeBitSet(optionals, 6);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetTid()) {
-          oprot.writeI64(struct.tid);
-        }
-        if (struct.isSetDir()) {
-          oprot.writeString(struct.dir);
-        }
-        if (struct.isSetFiles()) {
-          {
-            oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent, java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> _iter356 : struct.files.entrySet())
-            {
-              _iter356.getKey().write(oprot);
-              {
-                oprot.writeI32(_iter356.getValue().size());
-                for (java.util.Map.Entry<java.lang.String, org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _iter357 : _iter356.getValue().entrySet())
-                {
-                  oprot.writeString(_iter357.getKey());
-                  _iter357.getValue().write(oprot);
-                }
-              }
-            }
-          }
-        }
-        if (struct.isSetSetTime()) {
-          oprot.writeBool(struct.setTime);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, loadFiles_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.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.dir = iprot.readString();
-          struct.setDirIsSet(true);
-        }
-        if (incoming.get(4)) {
-          {
-            org.apache.thrift.protocol.TMap _map358 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.MAP); 
-            struct.files = new java.util.HashMap<org.apache.accumulo.core.dataImpl.thrift.TKeyExtent,java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>>(2*_map358.size);
-            @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent _key359;
-            @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> _val360;
-            for (int _i361 = 0; _i361 < _map358.size; ++_i361)
-            {
-              _key359 = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent();
-              _key359.read(iprot);
-              {
-                org.apache.thrift.protocol.TMap _map362 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT); 
-                _val360 = new java.util.HashMap<java.lang.String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>(2*_map362.size);
-                @org.apache.thrift.annotation.Nullable java.lang.String _key363;
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.MapFileInfo _val364;
-                for (int _i365 = 0; _i365 < _map362.size; ++_i365)
-                {
-                  _key363 = iprot.readString();
-                  _val364 = new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo();
-                  _val364.read(iprot);
-                  _val360.put(_key363, _val364);
-                }
-              }
-              struct.files.put(_key359, _val360);
-            }
-          }
-          struct.setFilesIsSet(true);
-        }
-        if (incoming.get(5)) {
-          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 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.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable 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.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.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public splitTablet_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 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.trace.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.trace.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.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.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 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, 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,
-      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 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 NotServingTabletException getNste() {
-      return this.nste;
-    }
-
-    public splitTablet_result setNste(@org.apache.thrift.annotation.Nullable 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((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 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 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 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.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String 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.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.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public loadTablet_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 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.trace.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.trace.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.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.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.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String 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.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.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public unloadTablet_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 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.trace.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.trace.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.tabletserver.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.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.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.tabletserver.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 flush_args implements org.apache.thrift.TBase<flush_args, flush_args._Fields>, java.io.Serializable, Cloneable, Comparable<flush_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("flush_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 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 TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)2);
-    private static final org.apache.thrift.protocol.TField START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)5);
-    private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (short)6);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new flush_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new flush_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String lock; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // required
-    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow; // required
-    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow; // 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"),
-      LOCK((short)3, "lock"),
-      TABLE_ID((short)2, "tableId"),
-      START_ROW((short)5, "startRow"),
-      END_ROW((short)6, "endRow");
-
-      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 3: // LOCK
-            return LOCK;
-          case 2: // TABLE_ID
-            return TABLE_ID;
-          case 5: // START_ROW
-            return START_ROW;
-          case 6: // END_ROW
-            return END_ROW;
-          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.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.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.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.START_ROW, new org.apache.thrift.meta_data.FieldMetaData("startRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", 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(flush_args.class, metaDataMap);
-    }
-
-    public flush_args() {
-    }
-
-    public flush_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String lock,
-      java.lang.String tableId,
-      java.nio.ByteBuffer startRow,
-      java.nio.ByteBuffer endRow)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.lock = lock;
-      this.tableId = tableId;
-      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
-      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public flush_args(flush_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetLock()) {
-        this.lock = other.lock;
-      }
-      if (other.isSetTableId()) {
-        this.tableId = other.tableId;
-      }
-      if (other.isSetStartRow()) {
-        this.startRow = org.apache.thrift.TBaseHelper.copyBinary(other.startRow);
-      }
-      if (other.isSetEndRow()) {
-        this.endRow = org.apache.thrift.TBaseHelper.copyBinary(other.endRow);
-      }
-    }
-
-    @Override
-    public flush_args deepCopy() {
-      return new flush_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.lock = null;
-      this.tableId = null;
-      this.startRow = null;
-      this.endRow = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public flush_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 flush_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 flush_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 java.lang.String getTableId() {
-      return this.tableId;
-    }
-
-    public flush_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 byte[] getStartRow() {
-      setStartRow(org.apache.thrift.TBaseHelper.rightSize(startRow));
-      return startRow == null ? null : startRow.array();
-    }
-
-    public java.nio.ByteBuffer bufferForStartRow() {
-      return org.apache.thrift.TBaseHelper.copyBinary(startRow);
-    }
-
-    public flush_args setStartRow(byte[] startRow) {
-      this.startRow = startRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(startRow.clone());
-      return this;
-    }
-
-    public flush_args setStartRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow) {
-      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
-      return this;
-    }
-
-    public void unsetStartRow() {
-      this.startRow = null;
-    }
-
-    /** Returns true if field startRow is set (has been assigned a value) and false otherwise */
-    public boolean isSetStartRow() {
-      return this.startRow != null;
-    }
-
-    public void setStartRowIsSet(boolean value) {
-      if (!value) {
-        this.startRow = null;
-      }
-    }
-
-    public byte[] getEndRow() {
-      setEndRow(org.apache.thrift.TBaseHelper.rightSize(endRow));
-      return endRow == null ? null : endRow.array();
-    }
-
-    public java.nio.ByteBuffer bufferForEndRow() {
-      return org.apache.thrift.TBaseHelper.copyBinary(endRow);
-    }
-
-    public flush_args setEndRow(byte[] endRow) {
-      this.endRow = endRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(endRow.clone());
-      return this;
-    }
-
-    public flush_args setEndRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow) {
-      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
-      return this;
-    }
-
-    public void unsetEndRow() {
-      this.endRow = null;
-    }
-
-    /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
-    public boolean isSetEndRow() {
-      return this.endRow != null;
-    }
-
-    public void setEndRowIsSet(boolean value) {
-      if (!value) {
-        this.endRow = 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.trace.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 TABLE_ID:
-        if (value == null) {
-          unsetTableId();
-        } else {
-          setTableId((java.lang.String)value);
-        }
-        break;
-
-      case START_ROW:
-        if (value == null) {
-          unsetStartRow();
-        } else {
-          if (value instanceof byte[]) {
-            setStartRow((byte[])value);
-          } else {
-            setStartRow((java.nio.ByteBuffer)value);
-          }
-        }
-        break;
-
-      case END_ROW:
-        if (value == null) {
-          unsetEndRow();
-        } else {
-          if (value instanceof byte[]) {
-            setEndRow((byte[])value);
-          } else {
-            setEndRow((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 LOCK:
-        return getLock();
-
-      case TABLE_ID:
-        return getTableId();
-
-      case START_ROW:
-        return getStartRow();
-
-      case END_ROW:
-        return getEndRow();
-
-      }
-      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 TABLE_ID:
-        return isSetTableId();
-      case START_ROW:
-        return isSetStartRow();
-      case END_ROW:
-        return isSetEndRow();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof flush_args)
-        return this.equals((flush_args)that);
-      return false;
-    }
-
-    public boolean equals(flush_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_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_startRow = true && this.isSetStartRow();
-      boolean that_present_startRow = true && that.isSetStartRow();
-      if (this_present_startRow || that_present_startRow) {
-        if (!(this_present_startRow && that_present_startRow))
-          return false;
-        if (!this.startRow.equals(that.startRow))
-          return false;
-      }
-
-      boolean this_present_endRow = true && this.isSetEndRow();
-      boolean that_present_endRow = true && that.isSetEndRow();
-      if (this_present_endRow || that_present_endRow) {
-        if (!(this_present_endRow && that_present_endRow))
-          return false;
-        if (!this.endRow.equals(that.endRow))
-          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 + ((isSetTableId()) ? 131071 : 524287);
-      if (isSetTableId())
-        hashCode = hashCode * 8191 + tableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetStartRow()) ? 131071 : 524287);
-      if (isSetStartRow())
-        hashCode = hashCode * 8191 + startRow.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
-      if (isSetEndRow())
-        hashCode = hashCode * 8191 + endRow.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(flush_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(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(isSetStartRow(), other.isSetStartRow());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetStartRow()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetEndRow(), other.isSetEndRow());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetEndRow()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
-        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("flush_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("tableId:");
-      if (this.tableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tableId);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("startRow:");
-      if (this.startRow == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.startRow, sb);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("endRow:");
-      if (this.endRow == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.endRow, 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();
-      }
-    }
-
-    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 flush_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public flush_argsStandardScheme getScheme() {
-        return new flush_argsStandardScheme();
-      }
-    }
-
-    private static class flush_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<flush_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, flush_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.trace.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: // 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: // 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: // START_ROW
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.startRow = iprot.readBinary();
-                struct.setStartRowIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // END_ROW
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.endRow = iprot.readBinary();
-                struct.setEndRowIsSet(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, flush_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.tableId != null) {
-          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.tableId);
-          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.startRow != null) {
-          oprot.writeFieldBegin(START_ROW_FIELD_DESC);
-          oprot.writeBinary(struct.startRow);
-          oprot.writeFieldEnd();
-        }
-        if (struct.endRow != null) {
-          oprot.writeFieldBegin(END_ROW_FIELD_DESC);
-          oprot.writeBinary(struct.endRow);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class flush_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public flush_argsTupleScheme getScheme() {
-        return new flush_argsTupleScheme();
-      }
-    }
-
-    private static class flush_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<flush_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, flush_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.isSetTableId()) {
-          optionals.set(3);
-        }
-        if (struct.isSetStartRow()) {
-          optionals.set(4);
-        }
-        if (struct.isSetEndRow()) {
-          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.isSetTableId()) {
-          oprot.writeString(struct.tableId);
-        }
-        if (struct.isSetStartRow()) {
-          oprot.writeBinary(struct.startRow);
-        }
-        if (struct.isSetEndRow()) {
-          oprot.writeBinary(struct.endRow);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, flush_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.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.lock = iprot.readString();
-          struct.setLockIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.tableId = iprot.readString();
-          struct.setTableIdIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.startRow = iprot.readBinary();
-          struct.setStartRowIsSet(true);
-        }
-        if (incoming.get(5)) {
-          struct.endRow = iprot.readBinary();
-          struct.setEndRowIsSet(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.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String 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.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.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.trace.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.trace.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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public flushTablet_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 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.trace.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.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            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.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.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 chop_args implements org.apache.thrift.TBase<chop_args, chop_args._Fields>, java.io.Serializable, Cloneable, Comparable<chop_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("chop_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 chop_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new chop_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String 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.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.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(chop_args.class, metaDataMap);
-    }
-
-    public chop_args() {
-    }
-
-    public chop_args(
-      org.apache.accumulo.core.trace.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 chop_args(chop_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetLock()) {
-        this.lock = other.lock;
-      }
-      if (other.isSetExtent()) {
-        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
-      }
-    }
-
-    @Override
-    public chop_args deepCopy() {
-      return new chop_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.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public chop_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 chop_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 chop_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 chop_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.trace.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 chop_args)
-        return this.equals((chop_args)that);
-      return false;
-    }
-
-    public boolean equals(chop_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(chop_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("chop_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 chop_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public chop_argsStandardScheme getScheme() {
-        return new chop_argsStandardScheme();
-      }
-    }
-
-    private static class chop_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<chop_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, chop_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            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, chop_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 chop_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public chop_argsTupleScheme getScheme() {
-        return new chop_argsTupleScheme();
-      }
-    }
-
-    private static class chop_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<chop_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, chop_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, chop_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.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.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 compact_args implements org.apache.thrift.TBase<compact_args, compact_args._Fields>, java.io.Serializable, Cloneable, Comparable<compact_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_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 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 START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)5);
-    private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (short)6);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compact_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compact_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String lock; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // required
-    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow; // required
-    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow; // 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"),
-      TABLE_ID((short)4, "tableId"),
-      START_ROW((short)5, "startRow"),
-      END_ROW((short)6, "endRow");
-
-      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: // TABLE_ID
-            return TABLE_ID;
-          case 5: // START_ROW
-            return START_ROW;
-          case 6: // END_ROW
-            return END_ROW;
-          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.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.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.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.START_ROW, new org.apache.thrift.meta_data.FieldMetaData("startRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", 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(compact_args.class, metaDataMap);
-    }
-
-    public compact_args() {
-    }
-
-    public compact_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String lock,
-      java.lang.String tableId,
-      java.nio.ByteBuffer startRow,
-      java.nio.ByteBuffer endRow)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.lock = lock;
-      this.tableId = tableId;
-      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
-      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public compact_args(compact_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetLock()) {
-        this.lock = other.lock;
-      }
-      if (other.isSetTableId()) {
-        this.tableId = other.tableId;
-      }
-      if (other.isSetStartRow()) {
-        this.startRow = org.apache.thrift.TBaseHelper.copyBinary(other.startRow);
-      }
-      if (other.isSetEndRow()) {
-        this.endRow = org.apache.thrift.TBaseHelper.copyBinary(other.endRow);
-      }
-    }
-
-    @Override
-    public compact_args deepCopy() {
-      return new compact_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.lock = null;
-      this.tableId = null;
-      this.startRow = null;
-      this.endRow = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public compact_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 compact_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 compact_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 java.lang.String getTableId() {
-      return this.tableId;
-    }
-
-    public compact_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 byte[] getStartRow() {
-      setStartRow(org.apache.thrift.TBaseHelper.rightSize(startRow));
-      return startRow == null ? null : startRow.array();
-    }
-
-    public java.nio.ByteBuffer bufferForStartRow() {
-      return org.apache.thrift.TBaseHelper.copyBinary(startRow);
-    }
-
-    public compact_args setStartRow(byte[] startRow) {
-      this.startRow = startRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(startRow.clone());
-      return this;
-    }
-
-    public compact_args setStartRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow) {
-      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
-      return this;
-    }
-
-    public void unsetStartRow() {
-      this.startRow = null;
-    }
-
-    /** Returns true if field startRow is set (has been assigned a value) and false otherwise */
-    public boolean isSetStartRow() {
-      return this.startRow != null;
-    }
-
-    public void setStartRowIsSet(boolean value) {
-      if (!value) {
-        this.startRow = null;
-      }
-    }
-
-    public byte[] getEndRow() {
-      setEndRow(org.apache.thrift.TBaseHelper.rightSize(endRow));
-      return endRow == null ? null : endRow.array();
-    }
-
-    public java.nio.ByteBuffer bufferForEndRow() {
-      return org.apache.thrift.TBaseHelper.copyBinary(endRow);
-    }
-
-    public compact_args setEndRow(byte[] endRow) {
-      this.endRow = endRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(endRow.clone());
-      return this;
-    }
-
-    public compact_args setEndRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow) {
-      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
-      return this;
-    }
-
-    public void unsetEndRow() {
-      this.endRow = null;
-    }
-
-    /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
-    public boolean isSetEndRow() {
-      return this.endRow != null;
-    }
-
-    public void setEndRowIsSet(boolean value) {
-      if (!value) {
-        this.endRow = 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.trace.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 TABLE_ID:
-        if (value == null) {
-          unsetTableId();
-        } else {
-          setTableId((java.lang.String)value);
-        }
-        break;
-
-      case START_ROW:
-        if (value == null) {
-          unsetStartRow();
-        } else {
-          if (value instanceof byte[]) {
-            setStartRow((byte[])value);
-          } else {
-            setStartRow((java.nio.ByteBuffer)value);
-          }
-        }
-        break;
-
-      case END_ROW:
-        if (value == null) {
-          unsetEndRow();
-        } else {
-          if (value instanceof byte[]) {
-            setEndRow((byte[])value);
-          } else {
-            setEndRow((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 LOCK:
-        return getLock();
-
-      case TABLE_ID:
-        return getTableId();
-
-      case START_ROW:
-        return getStartRow();
-
-      case END_ROW:
-        return getEndRow();
-
-      }
-      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 TABLE_ID:
-        return isSetTableId();
-      case START_ROW:
-        return isSetStartRow();
-      case END_ROW:
-        return isSetEndRow();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof compact_args)
-        return this.equals((compact_args)that);
-      return false;
-    }
-
-    public boolean equals(compact_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_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_startRow = true && this.isSetStartRow();
-      boolean that_present_startRow = true && that.isSetStartRow();
-      if (this_present_startRow || that_present_startRow) {
-        if (!(this_present_startRow && that_present_startRow))
-          return false;
-        if (!this.startRow.equals(that.startRow))
-          return false;
-      }
-
-      boolean this_present_endRow = true && this.isSetEndRow();
-      boolean that_present_endRow = true && that.isSetEndRow();
-      if (this_present_endRow || that_present_endRow) {
-        if (!(this_present_endRow && that_present_endRow))
-          return false;
-        if (!this.endRow.equals(that.endRow))
-          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 + ((isSetTableId()) ? 131071 : 524287);
-      if (isSetTableId())
-        hashCode = hashCode * 8191 + tableId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetStartRow()) ? 131071 : 524287);
-      if (isSetStartRow())
-        hashCode = hashCode * 8191 + startRow.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
-      if (isSetEndRow())
-        hashCode = hashCode * 8191 + endRow.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(compact_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(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(isSetStartRow(), other.isSetStartRow());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetStartRow()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetEndRow(), other.isSetEndRow());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetEndRow()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
-        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("compact_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("tableId:");
-      if (this.tableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tableId);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("startRow:");
-      if (this.startRow == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.startRow, sb);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("endRow:");
-      if (this.endRow == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.endRow, 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();
-      }
-    }
-
-    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 compact_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public compact_argsStandardScheme getScheme() {
-        return new compact_argsStandardScheme();
-      }
-    }
-
-    private static class compact_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compact_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, compact_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            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: // 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: // START_ROW
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.startRow = iprot.readBinary();
-                struct.setStartRowIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // END_ROW
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.endRow = iprot.readBinary();
-                struct.setEndRowIsSet(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, compact_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.tableId != null) {
-          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.tableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.startRow != null) {
-          oprot.writeFieldBegin(START_ROW_FIELD_DESC);
-          oprot.writeBinary(struct.startRow);
-          oprot.writeFieldEnd();
-        }
-        if (struct.endRow != null) {
-          oprot.writeFieldBegin(END_ROW_FIELD_DESC);
-          oprot.writeBinary(struct.endRow);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class compact_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public compact_argsTupleScheme getScheme() {
-        return new compact_argsTupleScheme();
-      }
-    }
-
-    private static class compact_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compact_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, compact_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.isSetTableId()) {
-          optionals.set(3);
-        }
-        if (struct.isSetStartRow()) {
-          optionals.set(4);
-        }
-        if (struct.isSetEndRow()) {
-          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.isSetTableId()) {
-          oprot.writeString(struct.tableId);
-        }
-        if (struct.isSetStartRow()) {
-          oprot.writeBinary(struct.startRow);
-        }
-        if (struct.isSetEndRow()) {
-          oprot.writeBinary(struct.endRow);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, compact_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.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.lock = iprot.readString();
-          struct.setLockIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.tableId = iprot.readString();
-          struct.setTableIdIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.startRow = iprot.readBinary();
-          struct.setStartRowIsSet(true);
-        }
-        if (incoming.get(5)) {
-          struct.endRow = iprot.readBinary();
-          struct.setEndRowIsSet(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 getTabletServerStatus_args implements org.apache.thrift.TBase<getTabletServerStatus_args, getTabletServerStatus_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletServerStatus_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletServerStatus_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)3);
-    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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTabletServerStatus_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletServerStatus_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)3, "tinfo"),
-      CREDENTIALS((short)1, "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 3: // TINFO
-            return TINFO;
-          case 1: // 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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTabletServerStatus_args.class, metaDataMap);
-    }
-
-    public getTabletServerStatus_args() {
-    }
-
-    public getTabletServerStatus_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getTabletServerStatus_args(getTabletServerStatus_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public getTabletServerStatus_args deepCopy() {
-      return new getTabletServerStatus_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public getTabletServerStatus_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 getTabletServerStatus_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 TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getTabletServerStatus_args)
-        return this.equals((getTabletServerStatus_args)that);
-      return false;
-    }
-
-    public boolean equals(getTabletServerStatus_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getTabletServerStatus_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @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("getTabletServerStatus_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getTabletServerStatus_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletServerStatus_argsStandardScheme getScheme() {
-        return new getTabletServerStatus_argsStandardScheme();
-      }
-    }
-
-    private static class getTabletServerStatus_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletServerStatus_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletServerStatus_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 3: // 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 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;
-            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, getTabletServerStatus_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();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getTabletServerStatus_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletServerStatus_argsTupleScheme getScheme() {
-        return new getTabletServerStatus_argsTupleScheme();
-      }
-    }
-
-    private static class getTabletServerStatus_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletServerStatus_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletServerStatus_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getTabletServerStatus_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getTabletServerStatus_result implements org.apache.thrift.TBase<getTabletServerStatus_result, getTabletServerStatus_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletServerStatus_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletServerStatus_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 getTabletServerStatus_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletServerStatus_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TabletServerStatus 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.master.thrift.TabletServerStatus.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(getTabletServerStatus_result.class, metaDataMap);
-    }
-
-    public getTabletServerStatus_result() {
-    }
-
-    public getTabletServerStatus_result(
-      org.apache.accumulo.core.master.thrift.TabletServerStatus 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 getTabletServerStatus_result(getTabletServerStatus_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.master.thrift.TabletServerStatus(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public getTabletServerStatus_result deepCopy() {
-      return new getTabletServerStatus_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.master.thrift.TabletServerStatus getSuccess() {
-      return this.success;
-    }
-
-    public getTabletServerStatus_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.master.thrift.TabletServerStatus 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 getTabletServerStatus_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.master.thrift.TabletServerStatus)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 getTabletServerStatus_result)
-        return this.equals((getTabletServerStatus_result)that);
-      return false;
-    }
-
-    public boolean equals(getTabletServerStatus_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(getTabletServerStatus_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("getTabletServerStatus_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 getTabletServerStatus_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletServerStatus_resultStandardScheme getScheme() {
-        return new getTabletServerStatus_resultStandardScheme();
-      }
-    }
-
-    private static class getTabletServerStatus_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletServerStatus_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletServerStatus_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.master.thrift.TabletServerStatus();
-                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, getTabletServerStatus_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 getTabletServerStatus_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletServerStatus_resultTupleScheme getScheme() {
-        return new getTabletServerStatus_resultTupleScheme();
-      }
-    }
-
-    private static class getTabletServerStatus_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletServerStatus_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletServerStatus_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, getTabletServerStatus_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.master.thrift.TabletServerStatus();
-          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 getTabletStats_args implements org.apache.thrift.TBase<getTabletStats_args, getTabletStats_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletStats_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletStats_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)3);
-    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 TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTabletStats_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletStats_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String tableId; // 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)3, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      TABLE_ID((short)2, "tableId");
-
-      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 3: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 2: // TABLE_ID
-            return TABLE_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
-    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      tmpMap.put(_Fields.TABLE_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)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTabletStats_args.class, metaDataMap);
-    }
-
-    public getTabletStats_args() {
-    }
-
-    public getTabletStats_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String tableId)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.tableId = tableId;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getTabletStats_args(getTabletStats_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetTableId()) {
-        this.tableId = other.tableId;
-      }
-    }
-
-    @Override
-    public getTabletStats_args deepCopy() {
-      return new getTabletStats_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.tableId = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public getTabletStats_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 getTabletStats_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 getTableId() {
-      return this.tableId;
-    }
-
-    public getTabletStats_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;
-      }
-    }
-
-    @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 TABLE_ID:
-        if (value == null) {
-          unsetTableId();
-        } else {
-          setTableId((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 TABLE_ID:
-        return getTableId();
-
-      }
-      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 TABLE_ID:
-        return isSetTableId();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getTabletStats_args)
-        return this.equals((getTabletStats_args)that);
-      return false;
-    }
-
-    public boolean equals(getTabletStats_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_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;
-      }
-
-      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 + ((isSetTableId()) ? 131071 : 524287);
-      if (isSetTableId())
-        hashCode = hashCode * 8191 + tableId.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getTabletStats_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(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;
-        }
-      }
-      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("getTabletStats_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("tableId:");
-      if (this.tableId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tableId);
-      }
-      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 getTabletStats_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletStats_argsStandardScheme getScheme() {
-        return new getTabletStats_argsStandardScheme();
-      }
-    }
-
-    private static class getTabletStats_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletStats_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletStats_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 3: // 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 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: // 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;
-            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, getTabletStats_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.tableId != null) {
-          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
-          oprot.writeString(struct.tableId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getTabletStats_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletStats_argsTupleScheme getScheme() {
-        return new getTabletStats_argsTupleScheme();
-      }
-    }
-
-    private static class getTabletStats_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletStats_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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.isSetTableId()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetTableId()) {
-          oprot.writeString(struct.tableId);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getTabletStats_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(3);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.tableId = iprot.readString();
-          struct.setTableIdIsSet(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 getTabletStats_result implements org.apache.thrift.TBase<getTabletStats_result, getTabletStats_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletStats_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletStats_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTabletStats_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletStats_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.util.List<TabletStats> 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TabletStats.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(getTabletStats_result.class, metaDataMap);
-    }
-
-    public getTabletStats_result() {
-    }
-
-    public getTabletStats_result(
-      java.util.List<TabletStats> 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 getTabletStats_result(getTabletStats_result other) {
-      if (other.isSetSuccess()) {
-        java.util.List<TabletStats> __this__success = new java.util.ArrayList<TabletStats>(other.success.size());
-        for (TabletStats other_element : other.success) {
-          __this__success.add(new TabletStats(other_element));
-        }
-        this.success = __this__success;
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public getTabletStats_result deepCopy() {
-      return new getTabletStats_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<TabletStats> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(TabletStats elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<TabletStats>();
-      }
-      this.success.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<TabletStats> getSuccess() {
-      return this.success;
-    }
-
-    public getTabletStats_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TabletStats> 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 getTabletStats_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.util.List<TabletStats>)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 getTabletStats_result)
-        return this.equals((getTabletStats_result)that);
-      return false;
-    }
-
-    public boolean equals(getTabletStats_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(getTabletStats_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("getTabletStats_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
-    }
-
-    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 getTabletStats_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletStats_resultStandardScheme getScheme() {
-        return new getTabletStats_resultStandardScheme();
-      }
-    }
-
-    private static class getTabletStats_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletStats_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletStats_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 _list366 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<TabletStats>(_list366.size);
-                  @org.apache.thrift.annotation.Nullable TabletStats _elem367;
-                  for (int _i368 = 0; _i368 < _list366.size; ++_i368)
-                  {
-                    _elem367 = new TabletStats();
-                    _elem367.read(iprot);
-                    struct.success.add(_elem367);
-                  }
-                  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 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, getTabletStats_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 (TabletStats _iter369 : struct.success)
-            {
-              _iter369.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getTabletStats_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getTabletStats_resultTupleScheme getScheme() {
-        return new getTabletStats_resultTupleScheme();
-      }
-    }
-
-    private static class getTabletStats_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletStats_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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.writeI32(struct.success.size());
-            for (TabletStats _iter370 : struct.success)
-            {
-              _iter370.write(oprot);
-            }
-          }
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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 _list371 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<TabletStats>(_list371.size);
-            @org.apache.thrift.annotation.Nullable TabletStats _elem372;
-            for (int _i373 = 0; _i373 < _list371.size; ++_i373)
-            {
-              _elem372 = new TabletStats();
-              _elem372.read(iprot);
-              struct.success.add(_elem372);
-            }
-          }
-          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 getHistoricalStats_args implements org.apache.thrift.TBase<getHistoricalStats_args, getHistoricalStats_args._Fields>, java.io.Serializable, Cloneable, Comparable<getHistoricalStats_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getHistoricalStats_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getHistoricalStats_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getHistoricalStats_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)2, "tinfo"),
-      CREDENTIALS((short)1, "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 2: // TINFO
-            return TINFO;
-          case 1: // 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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getHistoricalStats_args.class, metaDataMap);
-    }
-
-    public getHistoricalStats_args() {
-    }
-
-    public getHistoricalStats_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getHistoricalStats_args(getHistoricalStats_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public getHistoricalStats_args deepCopy() {
-      return new getHistoricalStats_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public getHistoricalStats_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 getHistoricalStats_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 TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getHistoricalStats_args)
-        return this.equals((getHistoricalStats_args)that);
-      return false;
-    }
-
-    public boolean equals(getHistoricalStats_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getHistoricalStats_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @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("getHistoricalStats_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getHistoricalStats_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getHistoricalStats_argsStandardScheme getScheme() {
-        return new getHistoricalStats_argsStandardScheme();
-      }
-    }
-
-    private static class getHistoricalStats_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getHistoricalStats_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getHistoricalStats_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 2: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 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;
-            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, getHistoricalStats_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();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getHistoricalStats_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getHistoricalStats_argsTupleScheme getScheme() {
-        return new getHistoricalStats_argsTupleScheme();
-      }
-    }
-
-    private static class getHistoricalStats_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getHistoricalStats_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getHistoricalStats_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getHistoricalStats_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getHistoricalStats_result implements org.apache.thrift.TBase<getHistoricalStats_result, getHistoricalStats_result._Fields>, java.io.Serializable, Cloneable, Comparable<getHistoricalStats_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getHistoricalStats_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 getHistoricalStats_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getHistoricalStats_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable TabletStats 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, TabletStats.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(getHistoricalStats_result.class, metaDataMap);
-    }
-
-    public getHistoricalStats_result() {
-    }
-
-    public getHistoricalStats_result(
-      TabletStats 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 getHistoricalStats_result(getHistoricalStats_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new TabletStats(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public getHistoricalStats_result deepCopy() {
-      return new getHistoricalStats_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TabletStats getSuccess() {
-      return this.success;
-    }
-
-    public getHistoricalStats_result setSuccess(@org.apache.thrift.annotation.Nullable TabletStats 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 getHistoricalStats_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((TabletStats)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 getHistoricalStats_result)
-        return this.equals((getHistoricalStats_result)that);
-      return false;
-    }
-
-    public boolean equals(getHistoricalStats_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(getHistoricalStats_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("getHistoricalStats_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 getHistoricalStats_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getHistoricalStats_resultStandardScheme getScheme() {
-        return new getHistoricalStats_resultStandardScheme();
-      }
-    }
-
-    private static class getHistoricalStats_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getHistoricalStats_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getHistoricalStats_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 TabletStats();
-                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, getHistoricalStats_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 getHistoricalStats_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getHistoricalStats_resultTupleScheme getScheme() {
-        return new getHistoricalStats_resultTupleScheme();
-      }
-    }
-
-    private static class getHistoricalStats_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getHistoricalStats_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getHistoricalStats_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, getHistoricalStats_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 TabletStats();
-          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 halt_args implements org.apache.thrift.TBase<halt_args, halt_args._Fields>, java.io.Serializable, Cloneable, Comparable<halt_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("halt_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)3);
-    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)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new halt_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new halt_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String lock; // 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)3, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      LOCK((short)2, "lock");
-
-      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 3: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 2: // LOCK
-            return LOCK;
-          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.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.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)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(halt_args.class, metaDataMap);
-    }
-
-    public halt_args() {
-    }
-
-    public halt_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String lock)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.lock = lock;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public halt_args(halt_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetLock()) {
-        this.lock = other.lock;
-      }
-    }
-
-    @Override
-    public halt_args deepCopy() {
-      return new halt_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.lock = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public halt_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 halt_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 halt_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;
-      }
-    }
-
-    @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 LOCK:
-        if (value == null) {
-          unsetLock();
-        } else {
-          setLock((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 LOCK:
-        return getLock();
-
-      }
-      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();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof halt_args)
-        return this.equals((halt_args)that);
-      return false;
-    }
-
-    public boolean equals(halt_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;
-      }
-
-      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();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(halt_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;
-        }
-      }
-      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("halt_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;
-      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 halt_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public halt_argsStandardScheme getScheme() {
-        return new halt_argsStandardScheme();
-      }
-    }
-
-    private static class halt_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<halt_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, halt_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 3: // 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 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: // 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;
-            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, halt_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.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 halt_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public halt_argsTupleScheme getScheme() {
-        return new halt_argsTupleScheme();
-      }
-    }
-
-    private static class halt_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<halt_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, halt_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);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetLock()) {
-          oprot.writeString(struct.lock);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, halt_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(3);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.lock = iprot.readString();
-          struct.setLockIsSet(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 halt_result implements org.apache.thrift.TBase<halt_result, halt_result._Fields>, java.io.Serializable, Cloneable, Comparable<halt_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("halt_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new halt_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new halt_resultTupleSchemeFactory();
-
-    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 {
-      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 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.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(halt_result.class, metaDataMap);
-    }
-
-    public halt_result() {
-    }
-
-    public halt_result(
-      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
-    {
-      this();
-      this.sec = sec;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public halt_result(halt_result other) {
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public halt_result deepCopy() {
-      return new halt_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.sec = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
-      return this.sec;
-    }
-
-    public halt_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 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 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 SEC:
-        return isSetSec();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof halt_result)
-        return this.equals((halt_result)that);
-      return false;
-    }
-
-    public boolean equals(halt_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;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
-      if (isSetSec())
-        hashCode = hashCode * 8191 + sec.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(halt_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;
-        }
-      }
-      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("halt_result(");
-      boolean first = true;
-
-      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 {
-        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 halt_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public halt_resultStandardScheme getScheme() {
-        return new halt_resultStandardScheme();
-      }
-    }
-
-    private static class halt_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<halt_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, halt_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;
-            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, halt_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();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class halt_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public halt_resultTupleScheme getScheme() {
-        return new halt_resultTupleScheme();
-      }
-    }
-
-    private static class halt_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<halt_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, halt_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);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, halt_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.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 fastHalt_args implements org.apache.thrift.TBase<fastHalt_args, fastHalt_args._Fields>, java.io.Serializable, Cloneable, Comparable<fastHalt_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fastHalt_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)3);
-    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)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new fastHalt_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new fastHalt_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String lock; // 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)3, "tinfo"),
-      CREDENTIALS((short)1, "credentials"),
-      LOCK((short)2, "lock");
-
-      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 3: // TINFO
-            return TINFO;
-          case 1: // CREDENTIALS
-            return CREDENTIALS;
-          case 2: // LOCK
-            return LOCK;
-          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.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.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)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fastHalt_args.class, metaDataMap);
-    }
-
-    public fastHalt_args() {
-    }
-
-    public fastHalt_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String lock)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.lock = lock;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public fastHalt_args(fastHalt_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetLock()) {
-        this.lock = other.lock;
-      }
-    }
-
-    @Override
-    public fastHalt_args deepCopy() {
-      return new fastHalt_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.lock = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public fastHalt_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 fastHalt_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 fastHalt_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;
-      }
-    }
-
-    @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 LOCK:
-        if (value == null) {
-          unsetLock();
-        } else {
-          setLock((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 LOCK:
-        return getLock();
-
-      }
-      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();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof fastHalt_args)
-        return this.equals((fastHalt_args)that);
-      return false;
-    }
-
-    public boolean equals(fastHalt_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;
-      }
-
-      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();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(fastHalt_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;
-        }
-      }
-      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("fastHalt_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;
-      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 fastHalt_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public fastHalt_argsStandardScheme getScheme() {
-        return new fastHalt_argsStandardScheme();
-      }
-    }
-
-    private static class fastHalt_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<fastHalt_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, fastHalt_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 3: // 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 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: // 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;
-            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, fastHalt_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.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 fastHalt_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public fastHalt_argsTupleScheme getScheme() {
-        return new fastHalt_argsTupleScheme();
-      }
-    }
-
-    private static class fastHalt_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<fastHalt_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, fastHalt_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);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetLock()) {
-          oprot.writeString(struct.lock);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, fastHalt_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(3);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.lock = iprot.readString();
-          struct.setLockIsSet(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 getActiveCompactions_args implements org.apache.thrift.TBase<getActiveCompactions_args, getActiveCompactions_args._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveCompactions_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveCompactions_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.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.securityImpl.thrift.TCredentials credentials; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)2, "tinfo"),
-      CREDENTIALS((short)1, "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 2: // TINFO
-            return TINFO;
-          case 1: // 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.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveCompactions_args.class, metaDataMap);
-    }
-
-    public getActiveCompactions_args() {
-    }
-
-    public getActiveCompactions_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getActiveCompactions_args(getActiveCompactions_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public getActiveCompactions_args deepCopy() {
-      return new getActiveCompactions_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public getActiveCompactions_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 getActiveCompactions_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 TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getActiveCompactions_args)
-        return this.equals((getActiveCompactions_args)that);
-      return false;
-    }
-
-    public boolean equals(getActiveCompactions_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getActiveCompactions_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @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("getActiveCompactions_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getActiveCompactions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveCompactions_argsStandardScheme getScheme() {
-        return new getActiveCompactions_argsStandardScheme();
-      }
-    }
-
-    private static class getActiveCompactions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveCompactions_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 2: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 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;
-            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, getActiveCompactions_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();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getActiveCompactions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveCompactions_argsTupleScheme getScheme() {
-        return new getActiveCompactions_argsTupleScheme();
-      }
-    }
-
-    private static class getActiveCompactions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveCompactions_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getActiveCompactions_result implements org.apache.thrift.TBase<getActiveCompactions_result, getActiveCompactions_result._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveCompactions_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveCompactions_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.util.List<ActiveCompaction> 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ActiveCompaction.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(getActiveCompactions_result.class, metaDataMap);
-    }
-
-    public getActiveCompactions_result() {
-    }
-
-    public getActiveCompactions_result(
-      java.util.List<ActiveCompaction> 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 getActiveCompactions_result(getActiveCompactions_result other) {
-      if (other.isSetSuccess()) {
-        java.util.List<ActiveCompaction> __this__success = new java.util.ArrayList<ActiveCompaction>(other.success.size());
-        for (ActiveCompaction other_element : other.success) {
-          __this__success.add(new ActiveCompaction(other_element));
-        }
-        this.success = __this__success;
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public getActiveCompactions_result deepCopy() {
-      return new getActiveCompactions_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<ActiveCompaction> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(ActiveCompaction elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<ActiveCompaction>();
-      }
-      this.success.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<ActiveCompaction> getSuccess() {
-      return this.success;
-    }
-
-    public getActiveCompactions_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<ActiveCompaction> 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 getActiveCompactions_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.util.List<ActiveCompaction>)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 getActiveCompactions_result)
-        return this.equals((getActiveCompactions_result)that);
-      return false;
-    }
-
-    public boolean equals(getActiveCompactions_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(getActiveCompactions_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("getActiveCompactions_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
-    }
-
-    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 getActiveCompactions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveCompactions_resultStandardScheme getScheme() {
-        return new getActiveCompactions_resultStandardScheme();
-      }
-    }
-
-    private static class getActiveCompactions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveCompactions_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_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 _list374 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<ActiveCompaction>(_list374.size);
-                  @org.apache.thrift.annotation.Nullable ActiveCompaction _elem375;
-                  for (int _i376 = 0; _i376 < _list374.size; ++_i376)
-                  {
-                    _elem375 = new ActiveCompaction();
-                    _elem375.read(iprot);
-                    struct.success.add(_elem375);
-                  }
-                  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 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, getActiveCompactions_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 (ActiveCompaction _iter377 : struct.success)
-            {
-              _iter377.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getActiveCompactions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveCompactions_resultTupleScheme getScheme() {
-        return new getActiveCompactions_resultTupleScheme();
-      }
-    }
-
-    private static class getActiveCompactions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveCompactions_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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.writeI32(struct.success.size());
-            for (ActiveCompaction _iter378 : struct.success)
-            {
-              _iter378.write(oprot);
-            }
-          }
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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 _list379 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<ActiveCompaction>(_list379.size);
-            @org.apache.thrift.annotation.Nullable ActiveCompaction _elem380;
-            for (int _i381 = 0; _i381 < _list379.size; ++_i381)
-            {
-              _elem380 = new ActiveCompaction();
-              _elem380.read(iprot);
-              struct.success.add(_elem380);
-            }
-          }
-          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 removeLogs_args implements org.apache.thrift.TBase<removeLogs_args, removeLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<removeLogs_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("removeLogs_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 FILENAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("filenames", org.apache.thrift.protocol.TType.LIST, (short)3);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeLogs_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeLogs_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> filenames; // 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"),
-      FILENAMES((short)3, "filenames");
-
-      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: // FILENAMES
-            return FILENAMES;
-          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.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.FILENAMES, new org.apache.thrift.meta_data.FieldMetaData("filenames", 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(removeLogs_args.class, metaDataMap);
-    }
-
-    public removeLogs_args() {
-    }
-
-    public removeLogs_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.util.List<java.lang.String> filenames)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.filenames = filenames;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public removeLogs_args(removeLogs_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetFilenames()) {
-        java.util.List<java.lang.String> __this__filenames = new java.util.ArrayList<java.lang.String>(other.filenames);
-        this.filenames = __this__filenames;
-      }
-    }
-
-    @Override
-    public removeLogs_args deepCopy() {
-      return new removeLogs_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.filenames = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public removeLogs_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 removeLogs_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 getFilenamesSize() {
-      return (this.filenames == null) ? 0 : this.filenames.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<java.lang.String> getFilenamesIterator() {
-      return (this.filenames == null) ? null : this.filenames.iterator();
-    }
-
-    public void addToFilenames(java.lang.String elem) {
-      if (this.filenames == null) {
-        this.filenames = new java.util.ArrayList<java.lang.String>();
-      }
-      this.filenames.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<java.lang.String> getFilenames() {
-      return this.filenames;
-    }
-
-    public removeLogs_args setFilenames(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> filenames) {
-      this.filenames = filenames;
-      return this;
-    }
-
-    public void unsetFilenames() {
-      this.filenames = null;
-    }
-
-    /** Returns true if field filenames is set (has been assigned a value) and false otherwise */
-    public boolean isSetFilenames() {
-      return this.filenames != null;
-    }
-
-    public void setFilenamesIsSet(boolean value) {
-      if (!value) {
-        this.filenames = 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.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case FILENAMES:
-        if (value == null) {
-          unsetFilenames();
-        } else {
-          setFilenames((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 CREDENTIALS:
-        return getCredentials();
-
-      case FILENAMES:
-        return getFilenames();
-
-      }
-      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 FILENAMES:
-        return isSetFilenames();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof removeLogs_args)
-        return this.equals((removeLogs_args)that);
-      return false;
-    }
-
-    public boolean equals(removeLogs_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_filenames = true && this.isSetFilenames();
-      boolean that_present_filenames = true && that.isSetFilenames();
-      if (this_present_filenames || that_present_filenames) {
-        if (!(this_present_filenames && that_present_filenames))
-          return false;
-        if (!this.filenames.equals(that.filenames))
-          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 + ((isSetFilenames()) ? 131071 : 524287);
-      if (isSetFilenames())
-        hashCode = hashCode * 8191 + filenames.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(removeLogs_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(isSetFilenames(), other.isSetFilenames());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetFilenames()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filenames, other.filenames);
-        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("removeLogs_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("filenames:");
-      if (this.filenames == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.filenames);
-      }
-      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 removeLogs_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public removeLogs_argsStandardScheme getScheme() {
-        return new removeLogs_argsStandardScheme();
-      }
-    }
-
-    private static class removeLogs_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<removeLogs_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, removeLogs_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // FILENAMES
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list382 = iprot.readListBegin();
-                  struct.filenames = new java.util.ArrayList<java.lang.String>(_list382.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem383;
-                  for (int _i384 = 0; _i384 < _list382.size; ++_i384)
-                  {
-                    _elem383 = iprot.readString();
-                    struct.filenames.add(_elem383);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setFilenamesIsSet(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, removeLogs_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.filenames != null) {
-          oprot.writeFieldBegin(FILENAMES_FIELD_DESC);
-          {
-            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filenames.size()));
-            for (java.lang.String _iter385 : struct.filenames)
-            {
-              oprot.writeString(_iter385);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class removeLogs_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public removeLogs_argsTupleScheme getScheme() {
-        return new removeLogs_argsTupleScheme();
-      }
-    }
-
-    private static class removeLogs_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<removeLogs_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, removeLogs_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.isSetFilenames()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetFilenames()) {
-          {
-            oprot.writeI32(struct.filenames.size());
-            for (java.lang.String _iter386 : struct.filenames)
-            {
-              oprot.writeString(_iter386);
-            }
-          }
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(3);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          {
-            org.apache.thrift.protocol.TList _list387 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.filenames = new java.util.ArrayList<java.lang.String>(_list387.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem388;
-            for (int _i389 = 0; _i389 < _list387.size; ++_i389)
-            {
-              _elem388 = iprot.readString();
-              struct.filenames.add(_elem388);
-            }
-          }
-          struct.setFilenamesIsSet(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 getActiveLogs_args implements org.apache.thrift.TBase<getActiveLogs_args, getActiveLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveLogs_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveLogs_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)1, "tinfo"),
-      CREDENTIALS((short)2, "credentials");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 1: // TINFO
-            return TINFO;
-          case 2: // CREDENTIALS
-            return CREDENTIALS;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @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.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveLogs_args.class, metaDataMap);
-    }
-
-    public getActiveLogs_args() {
-    }
-
-    public getActiveLogs_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getActiveLogs_args(getActiveLogs_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public getActiveLogs_args deepCopy() {
-      return new getActiveLogs_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public getActiveLogs_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 getActiveLogs_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 TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getActiveLogs_args)
-        return this.equals((getActiveLogs_args)that);
-      return false;
-    }
-
-    public boolean equals(getActiveLogs_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getActiveLogs_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @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("getActiveLogs_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getActiveLogs_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveLogs_argsStandardScheme getScheme() {
-        return new getActiveLogs_argsStandardScheme();
-      }
-    }
-
-    private static class getActiveLogs_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveLogs_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getActiveLogs_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getActiveLogs_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveLogs_argsTupleScheme getScheme() {
-        return new getActiveLogs_argsTupleScheme();
-      }
-    }
-
-    private static class getActiveLogs_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveLogs_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getActiveLogs_result implements org.apache.thrift.TBase<getActiveLogs_result, getActiveLogs_result._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveLogs_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveLogs_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @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))));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveLogs_result.class, metaDataMap);
-    }
-
-    public getActiveLogs_result() {
-    }
-
-    public getActiveLogs_result(
-      java.util.List<java.lang.String> success)
-    {
-      this();
-      this.success = success;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getActiveLogs_result(getActiveLogs_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;
-      }
-    }
-
-    @Override
-    public getActiveLogs_result deepCopy() {
-      return new getActiveLogs_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = 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 getActiveLogs_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;
-      }
-    }
-
-    @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;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @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 getActiveLogs_result)
-        return this.equals((getActiveLogs_result)that);
-      return false;
-    }
-
-    public boolean equals(getActiveLogs_result that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
-      if (isSetSuccess())
-        hashCode = hashCode * 8191 + success.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getActiveLogs_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("getActiveLogs_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getActiveLogs_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveLogs_resultStandardScheme getScheme() {
-        return new getActiveLogs_resultStandardScheme();
-      }
-    }
-
-    private static class getActiveLogs_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveLogs_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_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 _list390 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<java.lang.String>(_list390.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _elem391;
-                  for (int _i392 = 0; _i392 < _list390.size; ++_i392)
-                  {
-                    _elem391 = iprot.readString();
-                    struct.success.add(_elem391);
-                  }
-                  iprot.readListEnd();
-                }
-                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, getActiveLogs_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 _iter393 : struct.success)
-            {
-              oprot.writeString(_iter393);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getActiveLogs_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getActiveLogs_resultTupleScheme getScheme() {
-        return new getActiveLogs_resultTupleScheme();
-      }
-    }
-
-    private static class getActiveLogs_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveLogs_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          {
-            oprot.writeI32(struct.success.size());
-            for (java.lang.String _iter394 : struct.success)
-            {
-              oprot.writeString(_iter394);
-            }
-          }
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          {
-            org.apache.thrift.protocol.TList _list395 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
-            struct.success = new java.util.ArrayList<java.lang.String>(_list395.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _elem396;
-            for (int _i397 = 0; _i397 < _list395.size; ++_i397)
-            {
-              _elem396 = iprot.readString();
-              struct.success.add(_elem396);
-            }
-          }
-          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 startGetSummaries_args implements org.apache.thrift.TBase<startGetSummaries_args, startGetSummaries_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummaries_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummaries_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 REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummaries_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummaries_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request; // 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"),
-      REQUEST((short)3, "request");
-
-      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: // REQUEST
-            return REQUEST;
-          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.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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummaries_args.class, metaDataMap);
-    }
-
-    public startGetSummaries_args() {
-    }
-
-    public startGetSummaries_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.request = request;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startGetSummaries_args(startGetSummaries_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetRequest()) {
-        this.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest(other.request);
-      }
-    }
-
-    @Override
-    public startGetSummaries_args deepCopy() {
-      return new startGetSummaries_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.request = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startGetSummaries_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 startGetSummaries_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.TSummaryRequest getRequest() {
-      return this.request;
-    }
-
-    public startGetSummaries_args setRequest(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) {
-      this.request = request;
-      return this;
-    }
-
-    public void unsetRequest() {
-      this.request = null;
-    }
-
-    /** Returns true if field request is set (has been assigned a value) and false otherwise */
-    public boolean isSetRequest() {
-      return this.request != null;
-    }
-
-    public void setRequestIsSet(boolean value) {
-      if (!value) {
-        this.request = 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.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case REQUEST:
-        if (value == null) {
-          unsetRequest();
-        } else {
-          setRequest((org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest)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 REQUEST:
-        return getRequest();
-
-      }
-      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 REQUEST:
-        return isSetRequest();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof startGetSummaries_args)
-        return this.equals((startGetSummaries_args)that);
-      return false;
-    }
-
-    public boolean equals(startGetSummaries_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_request = true && this.isSetRequest();
-      boolean that_present_request = true && that.isSetRequest();
-      if (this_present_request || that_present_request) {
-        if (!(this_present_request && that_present_request))
-          return false;
-        if (!this.request.equals(that.request))
-          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 + ((isSetRequest()) ? 131071 : 524287);
-      if (isSetRequest())
-        hashCode = hashCode * 8191 + request.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(startGetSummaries_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(isSetRequest(), other.isSetRequest());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRequest()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
-        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("startGetSummaries_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("request:");
-      if (this.request == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.request);
-      }
-      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 (request != null) {
-        request.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 startGetSummaries_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummaries_argsStandardScheme getScheme() {
-        return new startGetSummaries_argsStandardScheme();
-      }
-    }
-
-    private static class startGetSummaries_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummaries_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummaries_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // REQUEST
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
-                struct.request.read(iprot);
-                struct.setRequestIsSet(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, startGetSummaries_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.request != null) {
-          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
-          struct.request.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class startGetSummaries_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummaries_argsTupleScheme getScheme() {
-        return new startGetSummaries_argsTupleScheme();
-      }
-    }
-
-    private static class startGetSummaries_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummaries_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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.isSetRequest()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetRequest()) {
-          struct.request.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(3);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
-          struct.request.read(iprot);
-          struct.setRequestIsSet(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 startGetSummaries_result implements org.apache.thrift.TBase<startGetSummaries_result, startGetSummaries_result._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummaries_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummaries_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.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 startGetSummaries_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummaries_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries success; // required
-    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.clientImpl.thrift.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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaries.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.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, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummaries_result.class, metaDataMap);
-    }
-
-    public startGetSummaries_result() {
-    }
-
-    public startGetSummaries_result(
-      org.apache.accumulo.core.dataImpl.thrift.TSummaries success,
-      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
-      org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException tope)
-    {
-      this();
-      this.success = success;
-      this.sec = sec;
-      this.tope = tope;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startGetSummaries_result(startGetSummaries_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-      if (other.isSetTope()) {
-        this.tope = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(other.tope);
-      }
-    }
-
-    @Override
-    public startGetSummaries_result deepCopy() {
-      return new startGetSummaries_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-      this.tope = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
-      return this.success;
-    }
-
-    public startGetSummaries_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummaries_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.clientImpl.thrift.ThriftTableOperationException getTope() {
-      return this.tope;
-    }
-
-    public startGetSummaries_result setTope(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.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((org.apache.accumulo.core.dataImpl.thrift.TSummaries)value);
-        }
-        break;
-
-      case SEC:
-        if (value == null) {
-          unsetSec();
-        } else {
-          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
-        }
-        break;
-
-      case TOPE:
-        if (value == null) {
-          unsetTope();
-        } else {
-          setTope((org.apache.accumulo.core.clientImpl.thrift.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 startGetSummaries_result)
-        return this.equals((startGetSummaries_result)that);
-      return false;
-    }
-
-    public boolean equals(startGetSummaries_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(startGetSummaries_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("startGetSummaries_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
-      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 startGetSummaries_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummaries_resultStandardScheme getScheme() {
-        return new startGetSummaries_resultStandardScheme();
-      }
-    }
-
-    private static class startGetSummaries_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummaries_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummaries_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.TSummaries();
-                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;
-            case 2: // TOPE
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tope = new org.apache.accumulo.core.clientImpl.thrift.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, startGetSummaries_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();
-        }
-        if (struct.tope != null) {
-          oprot.writeFieldBegin(TOPE_FIELD_DESC);
-          struct.tope.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class startGetSummaries_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummaries_resultTupleScheme getScheme() {
-        return new startGetSummaries_resultTupleScheme();
-      }
-    }
-
-    private static class startGetSummaries_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummaries_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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()) {
-          struct.success.write(oprot);
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-        if (struct.isSetTope()) {
-          struct.tope.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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)) {
-          struct.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries();
-          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);
-        }
-        if (incoming.get(2)) {
-          struct.tope = new org.apache.accumulo.core.clientImpl.thrift.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 startGetSummariesForPartition_args implements org.apache.thrift.TBase<startGetSummariesForPartition_args, startGetSummariesForPartition_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesForPartition_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesForPartition_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 REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField MODULUS_FIELD_DESC = new org.apache.thrift.protocol.TField("modulus", org.apache.thrift.protocol.TType.I32, (short)4);
-    private static final org.apache.thrift.protocol.TField REMAINDER_FIELD_DESC = new org.apache.thrift.protocol.TField("remainder", org.apache.thrift.protocol.TType.I32, (short)5);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummariesForPartition_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesForPartition_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request; // required
-    public int modulus; // required
-    public int remainder; // 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"),
-      REQUEST((short)3, "request"),
-      MODULUS((short)4, "modulus"),
-      REMAINDER((short)5, "remainder");
-
-      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: // REQUEST
-            return REQUEST;
-          case 4: // MODULUS
-            return MODULUS;
-          case 5: // REMAINDER
-            return REMAINDER;
-          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 __MODULUS_ISSET_ID = 0;
-    private static final int __REMAINDER_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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest.class)));
-      tmpMap.put(_Fields.MODULUS, new org.apache.thrift.meta_data.FieldMetaData("modulus", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      tmpMap.put(_Fields.REMAINDER, new org.apache.thrift.meta_data.FieldMetaData("remainder", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummariesForPartition_args.class, metaDataMap);
-    }
-
-    public startGetSummariesForPartition_args() {
-    }
-
-    public startGetSummariesForPartition_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request,
-      int modulus,
-      int remainder)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.request = request;
-      this.modulus = modulus;
-      setModulusIsSet(true);
-      this.remainder = remainder;
-      setRemainderIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startGetSummariesForPartition_args(startGetSummariesForPartition_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);
-      }
-      if (other.isSetRequest()) {
-        this.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest(other.request);
-      }
-      this.modulus = other.modulus;
-      this.remainder = other.remainder;
-    }
-
-    @Override
-    public startGetSummariesForPartition_args deepCopy() {
-      return new startGetSummariesForPartition_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.request = null;
-      setModulusIsSet(false);
-      this.modulus = 0;
-      setRemainderIsSet(false);
-      this.remainder = 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startGetSummariesForPartition_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 startGetSummariesForPartition_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.TSummaryRequest getRequest() {
-      return this.request;
-    }
-
-    public startGetSummariesForPartition_args setRequest(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) {
-      this.request = request;
-      return this;
-    }
-
-    public void unsetRequest() {
-      this.request = null;
-    }
-
-    /** Returns true if field request is set (has been assigned a value) and false otherwise */
-    public boolean isSetRequest() {
-      return this.request != null;
-    }
-
-    public void setRequestIsSet(boolean value) {
-      if (!value) {
-        this.request = null;
-      }
-    }
-
-    public int getModulus() {
-      return this.modulus;
-    }
-
-    public startGetSummariesForPartition_args setModulus(int modulus) {
-      this.modulus = modulus;
-      setModulusIsSet(true);
-      return this;
-    }
-
-    public void unsetModulus() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __MODULUS_ISSET_ID);
-    }
-
-    /** Returns true if field modulus is set (has been assigned a value) and false otherwise */
-    public boolean isSetModulus() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __MODULUS_ISSET_ID);
-    }
-
-    public void setModulusIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __MODULUS_ISSET_ID, value);
-    }
-
-    public int getRemainder() {
-      return this.remainder;
-    }
-
-    public startGetSummariesForPartition_args setRemainder(int remainder) {
-      this.remainder = remainder;
-      setRemainderIsSet(true);
-      return this;
-    }
-
-    public void unsetRemainder() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __REMAINDER_ISSET_ID);
-    }
-
-    /** Returns true if field remainder is set (has been assigned a value) and false otherwise */
-    public boolean isSetRemainder() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __REMAINDER_ISSET_ID);
-    }
-
-    public void setRemainderIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __REMAINDER_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 REQUEST:
-        if (value == null) {
-          unsetRequest();
-        } else {
-          setRequest((org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest)value);
-        }
-        break;
-
-      case MODULUS:
-        if (value == null) {
-          unsetModulus();
-        } else {
-          setModulus((java.lang.Integer)value);
-        }
-        break;
-
-      case REMAINDER:
-        if (value == null) {
-          unsetRemainder();
-        } else {
-          setRemainder((java.lang.Integer)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 REQUEST:
-        return getRequest();
-
-      case MODULUS:
-        return getModulus();
-
-      case REMAINDER:
-        return getRemainder();
-
-      }
-      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 REQUEST:
-        return isSetRequest();
-      case MODULUS:
-        return isSetModulus();
-      case REMAINDER:
-        return isSetRemainder();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof startGetSummariesForPartition_args)
-        return this.equals((startGetSummariesForPartition_args)that);
-      return false;
-    }
-
-    public boolean equals(startGetSummariesForPartition_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_request = true && this.isSetRequest();
-      boolean that_present_request = true && that.isSetRequest();
-      if (this_present_request || that_present_request) {
-        if (!(this_present_request && that_present_request))
-          return false;
-        if (!this.request.equals(that.request))
-          return false;
-      }
-
-      boolean this_present_modulus = true;
-      boolean that_present_modulus = true;
-      if (this_present_modulus || that_present_modulus) {
-        if (!(this_present_modulus && that_present_modulus))
-          return false;
-        if (this.modulus != that.modulus)
-          return false;
-      }
-
-      boolean this_present_remainder = true;
-      boolean that_present_remainder = true;
-      if (this_present_remainder || that_present_remainder) {
-        if (!(this_present_remainder && that_present_remainder))
-          return false;
-        if (this.remainder != that.remainder)
-          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 + ((isSetRequest()) ? 131071 : 524287);
-      if (isSetRequest())
-        hashCode = hashCode * 8191 + request.hashCode();
-
-      hashCode = hashCode * 8191 + modulus;
-
-      hashCode = hashCode * 8191 + remainder;
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(startGetSummariesForPartition_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(isSetRequest(), other.isSetRequest());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRequest()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetModulus(), other.isSetModulus());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetModulus()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.modulus, other.modulus);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetRemainder(), other.isSetRemainder());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRemainder()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remainder, other.remainder);
-        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("startGetSummariesForPartition_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("request:");
-      if (this.request == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.request);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("modulus:");
-      sb.append(this.modulus);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("remainder:");
-      sb.append(this.remainder);
-      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 (request != null) {
-        request.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 startGetSummariesForPartition_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesForPartition_argsStandardScheme getScheme() {
-        return new startGetSummariesForPartition_argsStandardScheme();
-      }
-    }
-
-    private static class startGetSummariesForPartition_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesForPartition_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesForPartition_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // REQUEST
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
-                struct.request.read(iprot);
-                struct.setRequestIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // MODULUS
-              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-                struct.modulus = iprot.readI32();
-                struct.setModulusIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // REMAINDER
-              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-                struct.remainder = iprot.readI32();
-                struct.setRemainderIsSet(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, startGetSummariesForPartition_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.request != null) {
-          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
-          struct.request.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(MODULUS_FIELD_DESC);
-        oprot.writeI32(struct.modulus);
-        oprot.writeFieldEnd();
-        oprot.writeFieldBegin(REMAINDER_FIELD_DESC);
-        oprot.writeI32(struct.remainder);
-        oprot.writeFieldEnd();
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class startGetSummariesForPartition_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesForPartition_argsTupleScheme getScheme() {
-        return new startGetSummariesForPartition_argsTupleScheme();
-      }
-    }
-
-    private static class startGetSummariesForPartition_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesForPartition_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesForPartition_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.isSetRequest()) {
-          optionals.set(2);
-        }
-        if (struct.isSetModulus()) {
-          optionals.set(3);
-        }
-        if (struct.isSetRemainder()) {
-          optionals.set(4);
-        }
-        oprot.writeBitSet(optionals, 5);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetRequest()) {
-          struct.request.write(oprot);
-        }
-        if (struct.isSetModulus()) {
-          oprot.writeI32(struct.modulus);
-        }
-        if (struct.isSetRemainder()) {
-          oprot.writeI32(struct.remainder);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesForPartition_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(5);
-        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.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
-          struct.request.read(iprot);
-          struct.setRequestIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.modulus = iprot.readI32();
-          struct.setModulusIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.remainder = iprot.readI32();
-          struct.setRemainderIsSet(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 startGetSummariesForPartition_result implements org.apache.thrift.TBase<startGetSummariesForPartition_result, startGetSummariesForPartition_result._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesForPartition_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesForPartition_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 startGetSummariesForPartition_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesForPartition_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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.TSummaries.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(startGetSummariesForPartition_result.class, metaDataMap);
-    }
-
-    public startGetSummariesForPartition_result() {
-    }
-
-    public startGetSummariesForPartition_result(
-      org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesForPartition_result(startGetSummariesForPartition_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public startGetSummariesForPartition_result deepCopy() {
-      return new startGetSummariesForPartition_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
-      return this.success;
-    }
-
-    public startGetSummariesForPartition_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesForPartition_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.TSummaries)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 startGetSummariesForPartition_result)
-        return this.equals((startGetSummariesForPartition_result)that);
-      return false;
-    }
-
-    public boolean equals(startGetSummariesForPartition_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(startGetSummariesForPartition_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("startGetSummariesForPartition_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 startGetSummariesForPartition_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesForPartition_resultStandardScheme getScheme() {
-        return new startGetSummariesForPartition_resultStandardScheme();
-      }
-    }
-
-    private static class startGetSummariesForPartition_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesForPartition_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesForPartition_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.TSummaries();
-                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, startGetSummariesForPartition_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 startGetSummariesForPartition_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesForPartition_resultTupleScheme getScheme() {
-        return new startGetSummariesForPartition_resultTupleScheme();
-      }
-    }
-
-    private static class startGetSummariesForPartition_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesForPartition_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesForPartition_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, startGetSummariesForPartition_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.TSummaries();
-          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 startGetSummariesFromFiles_args implements org.apache.thrift.TBase<startGetSummariesFromFiles_args, startGetSummariesFromFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesFromFiles_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesFromFiles_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 REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-    private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.MAP, (short)4);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummariesFromFiles_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesFromFiles_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request; // required
-    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files; // 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"),
-      REQUEST((short)3, "request"),
-      FILES((short)4, "files");
-
-      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: // REQUEST
-            return REQUEST;
-          case 4: // FILES
-            return FILES;
-          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.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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest.class)));
-      tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", 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.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.TRowRange.class)))));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummariesFromFiles_args.class, metaDataMap);
-    }
-
-    public startGetSummariesFromFiles_args() {
-    }
-
-    public startGetSummariesFromFiles_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request,
-      java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.request = request;
-      this.files = files;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public startGetSummariesFromFiles_args(startGetSummariesFromFiles_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetRequest()) {
-        this.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest(other.request);
-      }
-      if (other.isSetFiles()) {
-        java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> __this__files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>(other.files.size());
-        for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> other_element : other.files.entrySet()) {
-
-          java.lang.String other_element_key = other_element.getKey();
-          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> other_element_value = other_element.getValue();
-
-          java.lang.String __this__files_copy_key = other_element_key;
-
-          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> __this__files_copy_value = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRowRange>(other_element_value.size());
-          for (org.apache.accumulo.core.dataImpl.thrift.TRowRange other_element_value_element : other_element_value) {
-            __this__files_copy_value.add(new org.apache.accumulo.core.dataImpl.thrift.TRowRange(other_element_value_element));
-          }
-
-          __this__files.put(__this__files_copy_key, __this__files_copy_value);
-        }
-        this.files = __this__files;
-      }
-    }
-
-    @Override
-    public startGetSummariesFromFiles_args deepCopy() {
-      return new startGetSummariesFromFiles_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.request = null;
-      this.files = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public startGetSummariesFromFiles_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 startGetSummariesFromFiles_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.TSummaryRequest getRequest() {
-      return this.request;
-    }
-
-    public startGetSummariesFromFiles_args setRequest(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) {
-      this.request = request;
-      return this;
-    }
-
-    public void unsetRequest() {
-      this.request = null;
-    }
-
-    /** Returns true if field request is set (has been assigned a value) and false otherwise */
-    public boolean isSetRequest() {
-      return this.request != null;
-    }
-
-    public void setRequestIsSet(boolean value) {
-      if (!value) {
-        this.request = null;
-      }
-    }
-
-    public int getFilesSize() {
-      return (this.files == null) ? 0 : this.files.size();
-    }
-
-    public void putToFiles(java.lang.String key, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> val) {
-      if (this.files == null) {
-        this.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>();
-      }
-      this.files.put(key, val);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> getFiles() {
-      return this.files;
-    }
-
-    public startGetSummariesFromFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> 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;
-      }
-    }
-
-    @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 REQUEST:
-        if (value == null) {
-          unsetRequest();
-        } else {
-          setRequest((org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest)value);
-        }
-        break;
-
-      case FILES:
-        if (value == null) {
-          unsetFiles();
-        } else {
-          setFiles((java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>)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 REQUEST:
-        return getRequest();
-
-      case FILES:
-        return getFiles();
-
-      }
-      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 REQUEST:
-        return isSetRequest();
-      case FILES:
-        return isSetFiles();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof startGetSummariesFromFiles_args)
-        return this.equals((startGetSummariesFromFiles_args)that);
-      return false;
-    }
-
-    public boolean equals(startGetSummariesFromFiles_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_request = true && this.isSetRequest();
-      boolean that_present_request = true && that.isSetRequest();
-      if (this_present_request || that_present_request) {
-        if (!(this_present_request && that_present_request))
-          return false;
-        if (!this.request.equals(that.request))
-          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;
-      }
-
-      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 + ((isSetRequest()) ? 131071 : 524287);
-      if (isSetRequest())
-        hashCode = hashCode * 8191 + request.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
-      if (isSetFiles())
-        hashCode = hashCode * 8191 + files.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(startGetSummariesFromFiles_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(isSetRequest(), other.isSetRequest());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRequest()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
-        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;
-        }
-      }
-      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("startGetSummariesFromFiles_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("request:");
-      if (this.request == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.request);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("files:");
-      if (this.files == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.files);
-      }
-      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 (request != null) {
-        request.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 startGetSummariesFromFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesFromFiles_argsStandardScheme getScheme() {
-        return new startGetSummariesFromFiles_argsStandardScheme();
-      }
-    }
-
-    private static class startGetSummariesFromFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesFromFiles_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesFromFiles_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // REQUEST
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
-                struct.request.read(iprot);
-                struct.setRequestIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // FILES
-              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
-                {
-                  org.apache.thrift.protocol.TMap _map398 = iprot.readMapBegin();
-                  struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>(2*_map398.size);
-                  @org.apache.thrift.annotation.Nullable java.lang.String _key399;
-                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> _val400;
-                  for (int _i401 = 0; _i401 < _map398.size; ++_i401)
-                  {
-                    _key399 = iprot.readString();
-                    {
-                      org.apache.thrift.protocol.TList _list402 = iprot.readListBegin();
-                      _val400 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRowRange>(_list402.size);
-                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem403;
-                      for (int _i404 = 0; _i404 < _list402.size; ++_i404)
-                      {
-                        _elem403 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange();
-                        _elem403.read(iprot);
-                        _val400.add(_elem403);
-                      }
-                      iprot.readListEnd();
-                    }
-                    struct.files.put(_key399, _val400);
-                  }
-                  iprot.readMapEnd();
-                }
-                struct.setFilesIsSet(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, startGetSummariesFromFiles_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.request != null) {
-          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
-          struct.request.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.files != null) {
-          oprot.writeFieldBegin(FILES_FIELD_DESC);
-          {
-            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.files.size()));
-            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> _iter405 : struct.files.entrySet())
-            {
-              oprot.writeString(_iter405.getKey());
-              {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter405.getValue().size()));
-                for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter406 : _iter405.getValue())
-                {
-                  _iter406.write(oprot);
-                }
-                oprot.writeListEnd();
-              }
-            }
-            oprot.writeMapEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class startGetSummariesFromFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesFromFiles_argsTupleScheme getScheme() {
-        return new startGetSummariesFromFiles_argsTupleScheme();
-      }
-    }
-
-    private static class startGetSummariesFromFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesFromFiles_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFromFiles_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.isSetRequest()) {
-          optionals.set(2);
-        }
-        if (struct.isSetFiles()) {
-          optionals.set(3);
-        }
-        oprot.writeBitSet(optionals, 4);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetRequest()) {
-          struct.request.write(oprot);
-        }
-        if (struct.isSetFiles()) {
-          {
-            oprot.writeI32(struct.files.size());
-            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> _iter407 : struct.files.entrySet())
-            {
-              oprot.writeString(_iter407.getKey());
-              {
-                oprot.writeI32(_iter407.getValue().size());
-                for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter408 : _iter407.getValue())
-                {
-                  _iter408.write(oprot);
-                }
-              }
-            }
-          }
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFromFiles_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.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.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
-          struct.request.read(iprot);
-          struct.setRequestIsSet(true);
-        }
-        if (incoming.get(3)) {
-          {
-            org.apache.thrift.protocol.TMap _map409 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST); 
-            struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>(2*_map409.size);
-            @org.apache.thrift.annotation.Nullable java.lang.String _key410;
-            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> _val411;
-            for (int _i412 = 0; _i412 < _map409.size; ++_i412)
-            {
-              _key410 = iprot.readString();
-              {
-                org.apache.thrift.protocol.TList _list413 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-                _val411 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRowRange>(_list413.size);
-                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem414;
-                for (int _i415 = 0; _i415 < _list413.size; ++_i415)
-                {
-                  _elem414 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange();
-                  _elem414.read(iprot);
-                  _val411.add(_elem414);
-                }
-              }
-              struct.files.put(_key410, _val411);
-            }
-          }
-          struct.setFilesIsSet(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 startGetSummariesFromFiles_result implements org.apache.thrift.TBase<startGetSummariesFromFiles_result, startGetSummariesFromFiles_result._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesFromFiles_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesFromFiles_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 startGetSummariesFromFiles_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesFromFiles_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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.TSummaries.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(startGetSummariesFromFiles_result.class, metaDataMap);
-    }
-
-    public startGetSummariesFromFiles_result() {
-    }
-
-    public startGetSummariesFromFiles_result(
-      org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesFromFiles_result(startGetSummariesFromFiles_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public startGetSummariesFromFiles_result deepCopy() {
-      return new startGetSummariesFromFiles_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
-      return this.success;
-    }
-
-    public startGetSummariesFromFiles_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesFromFiles_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.TSummaries)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 startGetSummariesFromFiles_result)
-        return this.equals((startGetSummariesFromFiles_result)that);
-      return false;
-    }
-
-    public boolean equals(startGetSummariesFromFiles_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(startGetSummariesFromFiles_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("startGetSummariesFromFiles_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 startGetSummariesFromFiles_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesFromFiles_resultStandardScheme getScheme() {
-        return new startGetSummariesFromFiles_resultStandardScheme();
-      }
-    }
-
-    private static class startGetSummariesFromFiles_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesFromFiles_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesFromFiles_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.TSummaries();
-                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, startGetSummariesFromFiles_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 startGetSummariesFromFiles_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public startGetSummariesFromFiles_resultTupleScheme getScheme() {
-        return new startGetSummariesFromFiles_resultTupleScheme();
-      }
-    }
-
-    private static class startGetSummariesFromFiles_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesFromFiles_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFromFiles_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, startGetSummariesFromFiles_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.TSummaries();
-          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 contiuneGetSummaries_args implements org.apache.thrift.TBase<contiuneGetSummaries_args, contiuneGetSummaries_args._Fields>, java.io.Serializable, Cloneable, Comparable<contiuneGetSummaries_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("contiuneGetSummaries_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 SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.I64, (short)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new contiuneGetSummaries_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new contiuneGetSummaries_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public long sessionId; // 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"),
-      SESSION_ID((short)2, "sessionId");
-
-      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: // SESSION_ID
-            return SESSION_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 __SESSIONID_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.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", 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(contiuneGetSummaries_args.class, metaDataMap);
-    }
-
-    public contiuneGetSummaries_args() {
-    }
-
-    public contiuneGetSummaries_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      long sessionId)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.sessionId = sessionId;
-      setSessionIdIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public contiuneGetSummaries_args(contiuneGetSummaries_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      this.sessionId = other.sessionId;
-    }
-
-    @Override
-    public contiuneGetSummaries_args deepCopy() {
-      return new contiuneGetSummaries_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      setSessionIdIsSet(false);
-      this.sessionId = 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public contiuneGetSummaries_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;
-      }
-    }
-
-    public long getSessionId() {
-      return this.sessionId;
-    }
-
-    public contiuneGetSummaries_args setSessionId(long sessionId) {
-      this.sessionId = sessionId;
-      setSessionIdIsSet(true);
-      return this;
-    }
-
-    public void unsetSessionId() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSIONID_ISSET_ID);
-    }
-
-    /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */
-    public boolean isSetSessionId() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSIONID_ISSET_ID);
-    }
-
-    public void setSessionIdIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSIONID_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 SESSION_ID:
-        if (value == null) {
-          unsetSessionId();
-        } else {
-          setSessionId((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 SESSION_ID:
-        return getSessionId();
-
-      }
-      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 SESSION_ID:
-        return isSetSessionId();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof contiuneGetSummaries_args)
-        return this.equals((contiuneGetSummaries_args)that);
-      return false;
-    }
-
-    public boolean equals(contiuneGetSummaries_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_sessionId = true;
-      boolean that_present_sessionId = true;
-      if (this_present_sessionId || that_present_sessionId) {
-        if (!(this_present_sessionId && that_present_sessionId))
-          return false;
-        if (this.sessionId != that.sessionId)
-          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(sessionId);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(contiuneGetSummaries_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(isSetSessionId(), other.isSetSessionId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSessionId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, other.sessionId);
-        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("contiuneGetSummaries_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("sessionId:");
-      sb.append(this.sessionId);
-      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 contiuneGetSummaries_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public contiuneGetSummaries_argsStandardScheme getScheme() {
-        return new contiuneGetSummaries_argsStandardScheme();
-      }
-    }
-
-    private static class contiuneGetSummaries_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<contiuneGetSummaries_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, contiuneGetSummaries_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // SESSION_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.sessionId = iprot.readI64();
-                struct.setSessionIdIsSet(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, contiuneGetSummaries_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(SESSION_ID_FIELD_DESC);
-        oprot.writeI64(struct.sessionId);
-        oprot.writeFieldEnd();
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class contiuneGetSummaries_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public contiuneGetSummaries_argsTupleScheme getScheme() {
-        return new contiuneGetSummaries_argsTupleScheme();
-      }
-    }
-
-    private static class contiuneGetSummaries_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<contiuneGetSummaries_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, contiuneGetSummaries_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.isSetSessionId()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetSessionId()) {
-          oprot.writeI64(struct.sessionId);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, contiuneGetSummaries_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.sessionId = iprot.readI64();
-          struct.setSessionIdIsSet(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 contiuneGetSummaries_result implements org.apache.thrift.TBase<contiuneGetSummaries_result, contiuneGetSummaries_result._Fields>, java.io.Serializable, Cloneable, Comparable<contiuneGetSummaries_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("contiuneGetSummaries_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 contiuneGetSummaries_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new contiuneGetSummaries_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries success; // required
-    public @org.apache.thrift.annotation.Nullable 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.TSummaries.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, NoSuchScanIDException.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(contiuneGetSummaries_result.class, metaDataMap);
-    }
-
-    public contiuneGetSummaries_result() {
-    }
-
-    public contiuneGetSummaries_result(
-      org.apache.accumulo.core.dataImpl.thrift.TSummaries success,
-      NoSuchScanIDException nssi)
-    {
-      this();
-      this.success = success;
-      this.nssi = nssi;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public contiuneGetSummaries_result(contiuneGetSummaries_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
-      }
-      if (other.isSetNssi()) {
-        this.nssi = new NoSuchScanIDException(other.nssi);
-      }
-    }
-
-    @Override
-    public contiuneGetSummaries_result deepCopy() {
-      return new contiuneGetSummaries_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.nssi = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
-      return this.success;
-    }
-
-    public contiuneGetSummaries_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 NoSuchScanIDException getNssi() {
-      return this.nssi;
-    }
-
-    public contiuneGetSummaries_result setNssi(@org.apache.thrift.annotation.Nullable 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.TSummaries)value);
-        }
-        break;
-
-      case NSSI:
-        if (value == null) {
-          unsetNssi();
-        } else {
-          setNssi((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 contiuneGetSummaries_result)
-        return this.equals((contiuneGetSummaries_result)that);
-      return false;
-    }
-
-    public boolean equals(contiuneGetSummaries_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(contiuneGetSummaries_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("contiuneGetSummaries_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 contiuneGetSummaries_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public contiuneGetSummaries_resultStandardScheme getScheme() {
-        return new contiuneGetSummaries_resultStandardScheme();
-      }
-    }
-
-    private static class contiuneGetSummaries_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<contiuneGetSummaries_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, contiuneGetSummaries_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.TSummaries();
-                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 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, contiuneGetSummaries_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 contiuneGetSummaries_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public contiuneGetSummaries_resultTupleScheme getScheme() {
-        return new contiuneGetSummaries_resultTupleScheme();
-      }
-    }
-
-    private static class contiuneGetSummaries_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<contiuneGetSummaries_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, contiuneGetSummaries_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, contiuneGetSummaries_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.TSummaries();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.nssi = new 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 getCompactionQueueInfo_args implements org.apache.thrift.TBase<getCompactionQueueInfo_args, getCompactionQueueInfo_args._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionQueueInfo_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionQueueInfo_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionQueueInfo_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionQueueInfo_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)1, "tinfo"),
-      CREDENTIALS((short)2, "credentials");
-
-      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
-
-      static {
-        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 1: // TINFO
-            return TINFO;
-          case 2: // CREDENTIALS
-            return CREDENTIALS;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      @org.apache.thrift.annotation.Nullable
-      public static _Fields findByName(java.lang.String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final java.lang.String _fieldName;
-
-      _Fields(short thriftId, java.lang.String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      @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.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.securityImpl.thrift.TCredentials.class)));
-      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionQueueInfo_args.class, metaDataMap);
-    }
-
-    public getCompactionQueueInfo_args() {
-    }
-
-    public getCompactionQueueInfo_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getCompactionQueueInfo_args(getCompactionQueueInfo_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-    }
-
-    @Override
-    public getCompactionQueueInfo_args deepCopy() {
-      return new getCompactionQueueInfo_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public getCompactionQueueInfo_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 getCompactionQueueInfo_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 TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @Override
-    public java.lang.Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    @Override
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new java.lang.IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof getCompactionQueueInfo_args)
-        return this.equals((getCompactionQueueInfo_args)that);
-      return false;
-    }
-
-    public boolean equals(getCompactionQueueInfo_args that) {
-      if (that == null)
-        return false;
-      if (this == that)
-        return true;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int hashCode = 1;
-
-      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
-      if (isSetTinfo())
-        hashCode = hashCode * 8191 + tinfo.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
-      if (isSetCredentials())
-        hashCode = hashCode * 8191 + credentials.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(getCompactionQueueInfo_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    @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("getCompactionQueueInfo_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getCompactionQueueInfo_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getCompactionQueueInfo_argsStandardScheme getScheme() {
-        return new getCompactionQueueInfo_argsStandardScheme();
-      }
-    }
-
-    private static class getCompactionQueueInfo_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionQueueInfo_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionQueueInfo_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked in the validate method
-        struct.validate();
-      }
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getCompactionQueueInfo_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getCompactionQueueInfo_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getCompactionQueueInfo_argsTupleScheme getScheme() {
-        return new getCompactionQueueInfo_argsTupleScheme();
-      }
-    }
-
-    private static class getCompactionQueueInfo_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionQueueInfo_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet optionals = new java.util.BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
-        java.util.BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-      }
-    }
-
-    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
-      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
-    }
-  }
-
-  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
-  public static class getCompactionQueueInfo_result implements org.apache.thrift.TBase<getCompactionQueueInfo_result, getCompactionQueueInfo_result._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionQueueInfo_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionQueueInfo_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionQueueInfo_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionQueueInfo_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable java.util.List<TCompactionQueueSummary> 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCompactionQueueSummary.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(getCompactionQueueInfo_result.class, metaDataMap);
-    }
-
-    public getCompactionQueueInfo_result() {
-    }
-
-    public getCompactionQueueInfo_result(
-      java.util.List<TCompactionQueueSummary> 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 getCompactionQueueInfo_result(getCompactionQueueInfo_result other) {
-      if (other.isSetSuccess()) {
-        java.util.List<TCompactionQueueSummary> __this__success = new java.util.ArrayList<TCompactionQueueSummary>(other.success.size());
-        for (TCompactionQueueSummary other_element : other.success) {
-          __this__success.add(new TCompactionQueueSummary(other_element));
-        }
-        this.success = __this__success;
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public getCompactionQueueInfo_result deepCopy() {
-      return new getCompactionQueueInfo_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.Iterator<TCompactionQueueSummary> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(TCompactionQueueSummary elem) {
-      if (this.success == null) {
-        this.success = new java.util.ArrayList<TCompactionQueueSummary>();
-      }
-      this.success.add(elem);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.util.List<TCompactionQueueSummary> getSuccess() {
-      return this.success;
-    }
-
-    public getCompactionQueueInfo_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TCompactionQueueSummary> 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 getCompactionQueueInfo_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.util.List<TCompactionQueueSummary>)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 getCompactionQueueInfo_result)
-        return this.equals((getCompactionQueueInfo_result)that);
-      return false;
-    }
-
-    public boolean equals(getCompactionQueueInfo_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(getCompactionQueueInfo_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("getCompactionQueueInfo_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
-    }
-
-    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 getCompactionQueueInfo_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getCompactionQueueInfo_resultStandardScheme getScheme() {
-        return new getCompactionQueueInfo_resultStandardScheme();
-      }
-    }
-
-    private static class getCompactionQueueInfo_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionQueueInfo_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionQueueInfo_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 _list416 = iprot.readListBegin();
-                  struct.success = new java.util.ArrayList<TCompactionQueueSummary>(_list416.size);
-                  @org.apache.thrift.annotation.Nullable TCompactionQueueSummary _elem417;
-                  for (int _i418 = 0; _i418 < _list416.size; ++_i418)
-                  {
-                    _elem417 = new TCompactionQueueSummary();
-                    _elem417.read(iprot);
-                    struct.success.add(_elem417);
-                  }
-                  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 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, getCompactionQueueInfo_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 (TCompactionQueueSummary _iter419 : struct.success)
-            {
-              _iter419.write(oprot);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        if (struct.sec != null) {
-          oprot.writeFieldBegin(SEC_FIELD_DESC);
-          struct.sec.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getCompactionQueueInfo_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public getCompactionQueueInfo_resultTupleScheme getScheme() {
-        return new getCompactionQueueInfo_resultTupleScheme();
-      }
-    }
-
-    private static class getCompactionQueueInfo_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionQueueInfo_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_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.writeI32(struct.success.size());
-            for (TCompactionQueueSummary _iter420 : struct.success)
-            {
-              _iter420.write(oprot);
-            }
-          }
-        }
-        if (struct.isSetSec()) {
-          struct.sec.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_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 _list421 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
-            struct.success = new java.util.ArrayList<TCompactionQueueSummary>(_list421.size);
-            @org.apache.thrift.annotation.Nullable TCompactionQueueSummary _elem422;
-            for (int _i423 = 0; _i423 < _list421.size; ++_i423)
-            {
-              _elem422 = new TCompactionQueueSummary();
-              _elem422.read(iprot);
-              struct.success.add(_elem422);
-            }
-          }
-          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 reserveCompactionJob_args implements org.apache.thrift.TBase<reserveCompactionJob_args, reserveCompactionJob_args._Fields>, java.io.Serializable, Cloneable, Comparable<reserveCompactionJob_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("reserveCompactionJob_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 QUEUE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("queueName", org.apache.thrift.protocol.TType.STRING, (short)3);
-    private static final org.apache.thrift.protocol.TField PRIORITY_FIELD_DESC = new org.apache.thrift.protocol.TField("priority", org.apache.thrift.protocol.TType.I64, (short)4);
-    private static final org.apache.thrift.protocol.TField COMPACTOR_FIELD_DESC = new org.apache.thrift.protocol.TField("compactor", org.apache.thrift.protocol.TType.STRING, (short)5);
-    private static final org.apache.thrift.protocol.TField EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)6);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new reserveCompactionJob_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reserveCompactionJob_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String queueName; // required
-    public long priority; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // 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"),
-      QUEUE_NAME((short)3, "queueName"),
-      PRIORITY((short)4, "priority"),
-      COMPACTOR((short)5, "compactor"),
-      EXTERNAL_COMPACTION_ID((short)6, "externalCompactionId");
-
-      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: // QUEUE_NAME
-            return QUEUE_NAME;
-          case 4: // PRIORITY
-            return PRIORITY;
-          case 5: // COMPACTOR
-            return COMPACTOR;
-          case 6: // EXTERNAL_COMPACTION_ID
-            return EXTERNAL_COMPACTION_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 __PRIORITY_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.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.QUEUE_NAME, new org.apache.thrift.meta_data.FieldMetaData("queueName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.PRIORITY, new org.apache.thrift.meta_data.FieldMetaData("priority", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.COMPACTOR, new org.apache.thrift.meta_data.FieldMetaData("compactor", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", 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(reserveCompactionJob_args.class, metaDataMap);
-    }
-
-    public reserveCompactionJob_args() {
-    }
-
-    public reserveCompactionJob_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
-      java.lang.String queueName,
-      long priority,
-      java.lang.String compactor,
-      java.lang.String externalCompactionId)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.queueName = queueName;
-      this.priority = priority;
-      setPriorityIsSet(true);
-      this.compactor = compactor;
-      this.externalCompactionId = externalCompactionId;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public reserveCompactionJob_args(reserveCompactionJob_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);
-      }
-      if (other.isSetQueueName()) {
-        this.queueName = other.queueName;
-      }
-      this.priority = other.priority;
-      if (other.isSetCompactor()) {
-        this.compactor = other.compactor;
-      }
-      if (other.isSetExternalCompactionId()) {
-        this.externalCompactionId = other.externalCompactionId;
-      }
-    }
-
-    @Override
-    public reserveCompactionJob_args deepCopy() {
-      return new reserveCompactionJob_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.queueName = null;
-      setPriorityIsSet(false);
-      this.priority = 0;
-      this.compactor = null;
-      this.externalCompactionId = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public reserveCompactionJob_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 reserveCompactionJob_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 getQueueName() {
-      return this.queueName;
-    }
-
-    public reserveCompactionJob_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
-      this.queueName = queueName;
-      return this;
-    }
-
-    public void unsetQueueName() {
-      this.queueName = null;
-    }
-
-    /** Returns true if field queueName is set (has been assigned a value) and false otherwise */
-    public boolean isSetQueueName() {
-      return this.queueName != null;
-    }
-
-    public void setQueueNameIsSet(boolean value) {
-      if (!value) {
-        this.queueName = null;
-      }
-    }
-
-    public long getPriority() {
-      return this.priority;
-    }
-
-    public reserveCompactionJob_args setPriority(long priority) {
-      this.priority = priority;
-      setPriorityIsSet(true);
-      return this;
-    }
-
-    public void unsetPriority() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRIORITY_ISSET_ID);
-    }
-
-    /** Returns true if field priority is set (has been assigned a value) and false otherwise */
-    public boolean isSetPriority() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRIORITY_ISSET_ID);
-    }
-
-    public void setPriorityIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRIORITY_ISSET_ID, value);
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getCompactor() {
-      return this.compactor;
-    }
-
-    public reserveCompactionJob_args setCompactor(@org.apache.thrift.annotation.Nullable java.lang.String compactor) {
-      this.compactor = compactor;
-      return this;
-    }
-
-    public void unsetCompactor() {
-      this.compactor = null;
-    }
-
-    /** Returns true if field compactor is set (has been assigned a value) and false otherwise */
-    public boolean isSetCompactor() {
-      return this.compactor != null;
-    }
-
-    public void setCompactorIsSet(boolean value) {
-      if (!value) {
-        this.compactor = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public java.lang.String getExternalCompactionId() {
-      return this.externalCompactionId;
-    }
-
-    public reserveCompactionJob_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
-      this.externalCompactionId = externalCompactionId;
-      return this;
-    }
-
-    public void unsetExternalCompactionId() {
-      this.externalCompactionId = null;
-    }
-
-    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
-    public boolean isSetExternalCompactionId() {
-      return this.externalCompactionId != null;
-    }
-
-    public void setExternalCompactionIdIsSet(boolean value) {
-      if (!value) {
-        this.externalCompactionId = 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.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case QUEUE_NAME:
-        if (value == null) {
-          unsetQueueName();
-        } else {
-          setQueueName((java.lang.String)value);
-        }
-        break;
-
-      case PRIORITY:
-        if (value == null) {
-          unsetPriority();
-        } else {
-          setPriority((java.lang.Long)value);
-        }
-        break;
-
-      case COMPACTOR:
-        if (value == null) {
-          unsetCompactor();
-        } else {
-          setCompactor((java.lang.String)value);
-        }
-        break;
-
-      case EXTERNAL_COMPACTION_ID:
-        if (value == null) {
-          unsetExternalCompactionId();
-        } else {
-          setExternalCompactionId((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 QUEUE_NAME:
-        return getQueueName();
-
-      case PRIORITY:
-        return getPriority();
-
-      case COMPACTOR:
-        return getCompactor();
-
-      case EXTERNAL_COMPACTION_ID:
-        return getExternalCompactionId();
-
-      }
-      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 QUEUE_NAME:
-        return isSetQueueName();
-      case PRIORITY:
-        return isSetPriority();
-      case COMPACTOR:
-        return isSetCompactor();
-      case EXTERNAL_COMPACTION_ID:
-        return isSetExternalCompactionId();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof reserveCompactionJob_args)
-        return this.equals((reserveCompactionJob_args)that);
-      return false;
-    }
-
-    public boolean equals(reserveCompactionJob_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_queueName = true && this.isSetQueueName();
-      boolean that_present_queueName = true && that.isSetQueueName();
-      if (this_present_queueName || that_present_queueName) {
-        if (!(this_present_queueName && that_present_queueName))
-          return false;
-        if (!this.queueName.equals(that.queueName))
-          return false;
-      }
-
-      boolean this_present_priority = true;
-      boolean that_present_priority = true;
-      if (this_present_priority || that_present_priority) {
-        if (!(this_present_priority && that_present_priority))
-          return false;
-        if (this.priority != that.priority)
-          return false;
-      }
-
-      boolean this_present_compactor = true && this.isSetCompactor();
-      boolean that_present_compactor = true && that.isSetCompactor();
-      if (this_present_compactor || that_present_compactor) {
-        if (!(this_present_compactor && that_present_compactor))
-          return false;
-        if (!this.compactor.equals(that.compactor))
-          return false;
-      }
-
-      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
-      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
-      if (this_present_externalCompactionId || that_present_externalCompactionId) {
-        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
-          return false;
-        if (!this.externalCompactionId.equals(that.externalCompactionId))
-          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 + ((isSetQueueName()) ? 131071 : 524287);
-      if (isSetQueueName())
-        hashCode = hashCode * 8191 + queueName.hashCode();
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(priority);
-
-      hashCode = hashCode * 8191 + ((isSetCompactor()) ? 131071 : 524287);
-      if (isSetCompactor())
-        hashCode = hashCode * 8191 + compactor.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
-      if (isSetExternalCompactionId())
-        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(reserveCompactionJob_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(isSetQueueName(), other.isSetQueueName());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetQueueName()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetPriority(), other.isSetPriority());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetPriority()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.priority, other.priority);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetCompactor(), other.isSetCompactor());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCompactor()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactor, other.compactor);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetExternalCompactionId(), other.isSetExternalCompactionId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetExternalCompactionId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
-        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("reserveCompactionJob_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("queueName:");
-      if (this.queueName == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.queueName);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("priority:");
-      sb.append(this.priority);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("compactor:");
-      if (this.compactor == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.compactor);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("externalCompactionId:");
-      if (this.externalCompactionId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.externalCompactionId);
-      }
-      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 reserveCompactionJob_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public reserveCompactionJob_argsStandardScheme getScheme() {
-        return new reserveCompactionJob_argsStandardScheme();
-      }
-    }
-
-    private static class reserveCompactionJob_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<reserveCompactionJob_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, reserveCompactionJob_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // QUEUE_NAME
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.queueName = iprot.readString();
-                struct.setQueueNameIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 4: // PRIORITY
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.priority = iprot.readI64();
-                struct.setPriorityIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 5: // COMPACTOR
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.compactor = iprot.readString();
-                struct.setCompactorIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // EXTERNAL_COMPACTION_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.externalCompactionId = iprot.readString();
-                struct.setExternalCompactionIdIsSet(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, reserveCompactionJob_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.queueName != null) {
-          oprot.writeFieldBegin(QUEUE_NAME_FIELD_DESC);
-          oprot.writeString(struct.queueName);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(PRIORITY_FIELD_DESC);
-        oprot.writeI64(struct.priority);
-        oprot.writeFieldEnd();
-        if (struct.compactor != null) {
-          oprot.writeFieldBegin(COMPACTOR_FIELD_DESC);
-          oprot.writeString(struct.compactor);
-          oprot.writeFieldEnd();
-        }
-        if (struct.externalCompactionId != null) {
-          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
-          oprot.writeString(struct.externalCompactionId);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class reserveCompactionJob_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public reserveCompactionJob_argsTupleScheme getScheme() {
-        return new reserveCompactionJob_argsTupleScheme();
-      }
-    }
-
-    private static class reserveCompactionJob_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<reserveCompactionJob_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, reserveCompactionJob_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.isSetQueueName()) {
-          optionals.set(2);
-        }
-        if (struct.isSetPriority()) {
-          optionals.set(3);
-        }
-        if (struct.isSetCompactor()) {
-          optionals.set(4);
-        }
-        if (struct.isSetExternalCompactionId()) {
-          optionals.set(5);
-        }
-        oprot.writeBitSet(optionals, 6);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetQueueName()) {
-          oprot.writeString(struct.queueName);
-        }
-        if (struct.isSetPriority()) {
-          oprot.writeI64(struct.priority);
-        }
-        if (struct.isSetCompactor()) {
-          oprot.writeString(struct.compactor);
-        }
-        if (struct.isSetExternalCompactionId()) {
-          oprot.writeString(struct.externalCompactionId);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, reserveCompactionJob_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.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.queueName = iprot.readString();
-          struct.setQueueNameIsSet(true);
-        }
-        if (incoming.get(3)) {
-          struct.priority = iprot.readI64();
-          struct.setPriorityIsSet(true);
-        }
-        if (incoming.get(4)) {
-          struct.compactor = iprot.readString();
-          struct.setCompactorIsSet(true);
-        }
-        if (incoming.get(5)) {
-          struct.externalCompactionId = iprot.readString();
-          struct.setExternalCompactionIdIsSet(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 reserveCompactionJob_result implements org.apache.thrift.TBase<reserveCompactionJob_result, reserveCompactionJob_result._Fields>, java.io.Serializable, Cloneable, Comparable<reserveCompactionJob_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("reserveCompactionJob_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 reserveCompactionJob_resultStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reserveCompactionJob_resultTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable TExternalCompactionJob 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, TExternalCompactionJob.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(reserveCompactionJob_result.class, metaDataMap);
-    }
-
-    public reserveCompactionJob_result() {
-    }
-
-    public reserveCompactionJob_result(
-      TExternalCompactionJob 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 reserveCompactionJob_result(reserveCompactionJob_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new TExternalCompactionJob(other.success);
-      }
-      if (other.isSetSec()) {
-        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
-      }
-    }
-
-    @Override
-    public reserveCompactionJob_result deepCopy() {
-      return new reserveCompactionJob_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.sec = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public TExternalCompactionJob getSuccess() {
-      return this.success;
-    }
-
-    public reserveCompactionJob_result setSuccess(@org.apache.thrift.annotation.Nullable TExternalCompactionJob 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 reserveCompactionJob_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((TExternalCompactionJob)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 reserveCompactionJob_result)
-        return this.equals((reserveCompactionJob_result)that);
-      return false;
-    }
-
-    public boolean equals(reserveCompactionJob_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(reserveCompactionJob_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("reserveCompactionJob_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 reserveCompactionJob_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public reserveCompactionJob_resultStandardScheme getScheme() {
-        return new reserveCompactionJob_resultStandardScheme();
-      }
-    }
-
-    private static class reserveCompactionJob_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<reserveCompactionJob_result> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, reserveCompactionJob_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 TExternalCompactionJob();
-                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, reserveCompactionJob_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 reserveCompactionJob_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public reserveCompactionJob_resultTupleScheme getScheme() {
-        return new reserveCompactionJob_resultTupleScheme();
-      }
-    }
-
-    private static class reserveCompactionJob_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<reserveCompactionJob_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, reserveCompactionJob_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, reserveCompactionJob_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 TExternalCompactionJob();
-          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 compactionJobFinished_args implements org.apache.thrift.TBase<compactionJobFinished_args, compactionJobFinished_args._Fields>, java.io.Serializable, Cloneable, Comparable<compactionJobFinished_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionJobFinished_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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", 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.protocol.TField FILE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("fileSize", org.apache.thrift.protocol.TType.I64, (short)5);
-    private static final org.apache.thrift.protocol.TField ENTRIES_FIELD_DESC = new org.apache.thrift.protocol.TField("entries", org.apache.thrift.protocol.TType.I64, (short)6);
-
-    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionJobFinished_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionJobFinished_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
-    public long fileSize; // required
-    public long entries; // 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"),
-      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
-      EXTENT((short)4, "extent"),
-      FILE_SIZE((short)5, "fileSize"),
-      ENTRIES((short)6, "entries");
-
-      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: // EXTERNAL_COMPACTION_ID
-            return EXTERNAL_COMPACTION_ID;
-          case 4: // EXTENT
-            return EXTENT;
-          case 5: // FILE_SIZE
-            return FILE_SIZE;
-          case 6: // ENTRIES
-            return ENTRIES;
-          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 __FILESIZE_ISSET_ID = 0;
-    private static final int __ENTRIES_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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", 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.FILE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("fileSize", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-      tmpMap.put(_Fields.ENTRIES, new org.apache.thrift.meta_data.FieldMetaData("entries", 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(compactionJobFinished_args.class, metaDataMap);
-    }
-
-    public compactionJobFinished_args() {
-    }
-
-    public compactionJobFinished_args(
-      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,
-      long fileSize,
-      long entries)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.externalCompactionId = externalCompactionId;
-      this.extent = extent;
-      this.fileSize = fileSize;
-      setFileSizeIsSet(true);
-      this.entries = entries;
-      setEntriesIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public compactionJobFinished_args(compactionJobFinished_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);
-      }
-      if (other.isSetExternalCompactionId()) {
-        this.externalCompactionId = other.externalCompactionId;
-      }
-      if (other.isSetExtent()) {
-        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
-      }
-      this.fileSize = other.fileSize;
-      this.entries = other.entries;
-    }
-
-    @Override
-    public compactionJobFinished_args deepCopy() {
-      return new compactionJobFinished_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.externalCompactionId = null;
-      this.extent = null;
-      setFileSizeIsSet(false);
-      this.fileSize = 0;
-      setEntriesIsSet(false);
-      this.entries = 0;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public compactionJobFinished_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 compactionJobFinished_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 getExternalCompactionId() {
-      return this.externalCompactionId;
-    }
-
-    public compactionJobFinished_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
-      this.externalCompactionId = externalCompactionId;
-      return this;
-    }
-
-    public void unsetExternalCompactionId() {
-      this.externalCompactionId = null;
-    }
-
-    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
-    public boolean isSetExternalCompactionId() {
-      return this.externalCompactionId != null;
-    }
-
-    public void setExternalCompactionIdIsSet(boolean value) {
-      if (!value) {
-        this.externalCompactionId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
-      return this.extent;
-    }
-
-    public compactionJobFinished_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 long getFileSize() {
-      return this.fileSize;
-    }
-
-    public compactionJobFinished_args setFileSize(long fileSize) {
-      this.fileSize = fileSize;
-      setFileSizeIsSet(true);
-      return this;
-    }
-
-    public void unsetFileSize() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESIZE_ISSET_ID);
-    }
-
-    /** Returns true if field fileSize is set (has been assigned a value) and false otherwise */
-    public boolean isSetFileSize() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESIZE_ISSET_ID);
-    }
-
-    public void setFileSizeIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESIZE_ISSET_ID, value);
-    }
-
-    public long getEntries() {
-      return this.entries;
-    }
-
-    public compactionJobFinished_args setEntries(long entries) {
-      this.entries = entries;
-      setEntriesIsSet(true);
-      return this;
-    }
-
-    public void unsetEntries() {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ENTRIES_ISSET_ID);
-    }
-
-    /** Returns true if field entries is set (has been assigned a value) and false otherwise */
-    public boolean isSetEntries() {
-      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ENTRIES_ISSET_ID);
-    }
-
-    public void setEntriesIsSet(boolean value) {
-      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ENTRIES_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 EXTERNAL_COMPACTION_ID:
-        if (value == null) {
-          unsetExternalCompactionId();
-        } else {
-          setExternalCompactionId((java.lang.String)value);
-        }
-        break;
-
-      case EXTENT:
-        if (value == null) {
-          unsetExtent();
-        } else {
-          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
-        }
-        break;
-
-      case FILE_SIZE:
-        if (value == null) {
-          unsetFileSize();
-        } else {
-          setFileSize((java.lang.Long)value);
-        }
-        break;
-
-      case ENTRIES:
-        if (value == null) {
-          unsetEntries();
-        } else {
-          setEntries((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 EXTERNAL_COMPACTION_ID:
-        return getExternalCompactionId();
-
-      case EXTENT:
-        return getExtent();
-
-      case FILE_SIZE:
-        return getFileSize();
-
-      case ENTRIES:
-        return getEntries();
-
-      }
-      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 EXTERNAL_COMPACTION_ID:
-        return isSetExternalCompactionId();
-      case EXTENT:
-        return isSetExtent();
-      case FILE_SIZE:
-        return isSetFileSize();
-      case ENTRIES:
-        return isSetEntries();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof compactionJobFinished_args)
-        return this.equals((compactionJobFinished_args)that);
-      return false;
-    }
-
-    public boolean equals(compactionJobFinished_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_externalCompactionId = true && this.isSetExternalCompactionId();
-      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
-      if (this_present_externalCompactionId || that_present_externalCompactionId) {
-        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
-          return false;
-        if (!this.externalCompactionId.equals(that.externalCompactionId))
-          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_fileSize = true;
-      boolean that_present_fileSize = true;
-      if (this_present_fileSize || that_present_fileSize) {
-        if (!(this_present_fileSize && that_present_fileSize))
-          return false;
-        if (this.fileSize != that.fileSize)
-          return false;
-      }
-
-      boolean this_present_entries = true;
-      boolean that_present_entries = true;
-      if (this_present_entries || that_present_entries) {
-        if (!(this_present_entries && that_present_entries))
-          return false;
-        if (this.entries != that.entries)
-          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 + ((isSetExternalCompactionId()) ? 131071 : 524287);
-      if (isSetExternalCompactionId())
-        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
-      if (isSetExtent())
-        hashCode = hashCode * 8191 + extent.hashCode();
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(fileSize);
-
-      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(entries);
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(compactionJobFinished_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(isSetExternalCompactionId(), other.isSetExternalCompactionId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetExternalCompactionId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
-        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(isSetFileSize(), other.isSetFileSize());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetFileSize()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileSize, other.fileSize);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = java.lang.Boolean.compare(isSetEntries(), other.isSetEntries());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetEntries()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.entries, other.entries);
-        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("compactionJobFinished_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("externalCompactionId:");
-      if (this.externalCompactionId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.externalCompactionId);
-      }
-      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("fileSize:");
-      sb.append(this.fileSize);
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("entries:");
-      sb.append(this.entries);
-      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 compactionJobFinished_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public compactionJobFinished_argsStandardScheme getScheme() {
-        return new compactionJobFinished_argsStandardScheme();
-      }
-    }
-
-    private static class compactionJobFinished_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionJobFinished_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionJobFinished_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // EXTERNAL_COMPACTION_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.externalCompactionId = iprot.readString();
-                struct.setExternalCompactionIdIsSet(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;
-            case 5: // FILE_SIZE
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.fileSize = iprot.readI64();
-                struct.setFileSizeIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 6: // ENTRIES
-              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-                struct.entries = iprot.readI64();
-                struct.setEntriesIsSet(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, compactionJobFinished_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.externalCompactionId != null) {
-          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
-          oprot.writeString(struct.externalCompactionId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.extent != null) {
-          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
-          struct.extent.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldBegin(FILE_SIZE_FIELD_DESC);
-        oprot.writeI64(struct.fileSize);
-        oprot.writeFieldEnd();
-        oprot.writeFieldBegin(ENTRIES_FIELD_DESC);
-        oprot.writeI64(struct.entries);
-        oprot.writeFieldEnd();
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class compactionJobFinished_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public compactionJobFinished_argsTupleScheme getScheme() {
-        return new compactionJobFinished_argsTupleScheme();
-      }
-    }
-
-    private static class compactionJobFinished_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionJobFinished_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, compactionJobFinished_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.isSetExternalCompactionId()) {
-          optionals.set(2);
-        }
-        if (struct.isSetExtent()) {
-          optionals.set(3);
-        }
-        if (struct.isSetFileSize()) {
-          optionals.set(4);
-        }
-        if (struct.isSetEntries()) {
-          optionals.set(5);
-        }
-        oprot.writeBitSet(optionals, 6);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetExternalCompactionId()) {
-          oprot.writeString(struct.externalCompactionId);
-        }
-        if (struct.isSetExtent()) {
-          struct.extent.write(oprot);
-        }
-        if (struct.isSetFileSize()) {
-          oprot.writeI64(struct.fileSize);
-        }
-        if (struct.isSetEntries()) {
-          oprot.writeI64(struct.entries);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, compactionJobFinished_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.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.externalCompactionId = iprot.readString();
-          struct.setExternalCompactionIdIsSet(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.fileSize = iprot.readI64();
-          struct.setFileSizeIsSet(true);
-        }
-        if (incoming.get(5)) {
-          struct.entries = iprot.readI64();
-          struct.setEntriesIsSet(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 compactionJobFailed_args implements org.apache.thrift.TBase<compactionJobFailed_args, compactionJobFailed_args._Fields>, java.io.Serializable, Cloneable, Comparable<compactionJobFailed_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionJobFailed_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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", 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 compactionJobFailed_argsStandardSchemeFactory();
-    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionJobFailed_argsTupleSchemeFactory();
-
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials; // required
-    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // 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"),
-      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
-      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: // EXTERNAL_COMPACTION_ID
-            return EXTERNAL_COMPACTION_ID;
-          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.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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", 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(compactionJobFailed_args.class, metaDataMap);
-    }
-
-    public compactionJobFailed_args() {
-    }
-
-    public compactionJobFailed_args(
-      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)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.externalCompactionId = externalCompactionId;
-      this.extent = extent;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public compactionJobFailed_args(compactionJobFailed_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetExternalCompactionId()) {
-        this.externalCompactionId = other.externalCompactionId;
-      }
-      if (other.isSetExtent()) {
-        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
-      }
-    }
-
-    @Override
-    public compactionJobFailed_args deepCopy() {
-      return new compactionJobFailed_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.externalCompactionId = null;
-      this.extent = null;
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public compactionJobFailed_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 compactionJobFailed_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 getExternalCompactionId() {
-      return this.externalCompactionId;
-    }
-
-    public compactionJobFailed_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
-      this.externalCompactionId = externalCompactionId;
-      return this;
-    }
-
-    public void unsetExternalCompactionId() {
-      this.externalCompactionId = null;
-    }
-
-    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
-    public boolean isSetExternalCompactionId() {
-      return this.externalCompactionId != null;
-    }
-
-    public void setExternalCompactionIdIsSet(boolean value) {
-      if (!value) {
-        this.externalCompactionId = null;
-      }
-    }
-
-    @org.apache.thrift.annotation.Nullable
-    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
-      return this.extent;
-    }
-
-    public compactionJobFailed_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.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.securityImpl.thrift.TCredentials)value);
-        }
-        break;
-
-      case EXTERNAL_COMPACTION_ID:
-        if (value == null) {
-          unsetExternalCompactionId();
-        } else {
-          setExternalCompactionId((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 EXTERNAL_COMPACTION_ID:
-        return getExternalCompactionId();
-
-      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 EXTERNAL_COMPACTION_ID:
-        return isSetExternalCompactionId();
-      case EXTENT:
-        return isSetExtent();
-      }
-      throw new java.lang.IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(java.lang.Object that) {
-      if (that instanceof compactionJobFailed_args)
-        return this.equals((compactionJobFailed_args)that);
-      return false;
-    }
-
-    public boolean equals(compactionJobFailed_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_externalCompactionId = true && this.isSetExternalCompactionId();
-      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
-      if (this_present_externalCompactionId || that_present_externalCompactionId) {
-        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
-          return false;
-        if (!this.externalCompactionId.equals(that.externalCompactionId))
-          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 + ((isSetExternalCompactionId()) ? 131071 : 524287);
-      if (isSetExternalCompactionId())
-        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
-
-      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
-      if (isSetExtent())
-        hashCode = hashCode * 8191 + extent.hashCode();
-
-      return hashCode;
-    }
-
-    @Override
-    public int compareTo(compactionJobFailed_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(isSetExternalCompactionId(), other.isSetExternalCompactionId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetExternalCompactionId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
-        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("compactionJobFailed_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("externalCompactionId:");
-      if (this.externalCompactionId == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.externalCompactionId);
-      }
-      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 compactionJobFailed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public compactionJobFailed_argsStandardScheme getScheme() {
-        return new compactionJobFailed_argsStandardScheme();
-      }
-    }
-
-    private static class compactionJobFailed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionJobFailed_args> {
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionJobFailed_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.securityImpl.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // EXTERNAL_COMPACTION_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.externalCompactionId = iprot.readString();
-                struct.setExternalCompactionIdIsSet(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, compactionJobFailed_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.externalCompactionId != null) {
-          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
-          oprot.writeString(struct.externalCompactionId);
-          oprot.writeFieldEnd();
-        }
-        if (struct.extent != null) {
-          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
-          struct.extent.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class compactionJobFailed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
-      @Override
-      public compactionJobFailed_argsTupleScheme getScheme() {
-        return new compactionJobFailed_argsTupleScheme();
-      }
-    }
-
-    private static class compactionJobFailed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionJobFailed_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, compactionJobFailed_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.isSetExternalCompactionId()) {
-          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.isSetExternalCompactionId()) {
-          oprot.writeString(struct.externalCompactionId);
-        }
-        if (struct.isSetExtent()) {
-          struct.extent.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, compactionJobFailed_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.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.externalCompactionId = iprot.readString();
-          struct.setExternalCompactionIdIsSet(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/TabletServerClientService.java b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletServerClientService.java
new file mode 100644
index 0000000..e627c4f
--- /dev/null
+++ b/core/src/main/thrift-gen-java/org/apache/accumulo/core/tabletserver/thrift/TabletServerClientService.java
@@ -0,0 +1,21053 @@
+/*
+ * 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.tabletserver.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+public class TabletServerClientService {
+
+  public interface Iface {
+
+    public void flush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException;
+
+    public void compact(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.manager.thrift.TabletServerStatus getTabletServerStatus(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<TabletStats> getTabletStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public TabletStats getHistoricalStats(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 halt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public void fastHalt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException;
+
+    public java.util.List<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 removeLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException;
+
+    public java.util.List<java.lang.String> getActiveLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesForPartition(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesFromFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException;
+
+    public java.util.List<TCompactionQueueSummary> getCompactionQueueInfo(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 TExternalCompactionJob reserveCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+
+    public void compactionJobFinished(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, long fileSize, long entries) throws org.apache.thrift.TException;
+
+    public void compactionJobFailed(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 interface AsyncIface {
+
+    public void flush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void compact(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void getTabletServerStatus(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.manager.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException;
+
+    public void getTabletStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException;
+
+    public void getHistoricalStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException;
+
+    public void halt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void fastHalt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> 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<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException;
+
+    public void removeLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void getActiveLogs(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 startGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void startGetSummariesForPartition(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void startGetSummariesFromFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void contiuneGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException;
+
+    public void getCompactionQueueInfo(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException;
+
+    public void reserveCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException;
+
+    public void compactionJobFinished(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, long fileSize, long entries, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void compactionJobFailed(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 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 flush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
+    {
+      send_flush(tinfo, credentials, lock, tableId, startRow, endRow);
+    }
+
+    public void send_flush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
+    {
+      flush_args args = new flush_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      args.setTableId(tableId);
+      args.setStartRow(startRow);
+      args.setEndRow(endRow);
+      sendBaseOneway("flush", args);
+    }
+
+    @Override
+    public void compact(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
+    {
+      send_compact(tinfo, credentials, lock, tableId, startRow, endRow);
+    }
+
+    public void send_compact(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow) throws org.apache.thrift.TException
+    {
+      compact_args args = new compact_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      args.setTableId(tableId);
+      args.setStartRow(startRow);
+      args.setEndRow(endRow);
+      sendBaseOneway("compact", args);
+    }
+
+    @Override
+    public org.apache.accumulo.core.manager.thrift.TabletServerStatus getTabletServerStatus(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_getTabletServerStatus(tinfo, credentials);
+      return recv_getTabletServerStatus();
+    }
+
+    public void send_getTabletServerStatus(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getTabletServerStatus_args args = new getTabletServerStatus_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getTabletServerStatus", args);
+    }
+
+    public org.apache.accumulo.core.manager.thrift.TabletServerStatus recv_getTabletServerStatus() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getTabletServerStatus_result result = new getTabletServerStatus_result();
+      receiveBase(result, "getTabletServerStatus");
+      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, "getTabletServerStatus failed: unknown result");
+    }
+
+    @Override
+    public java.util.List<TabletStats> getTabletStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_getTabletStats(tinfo, credentials, tableId);
+      return recv_getTabletStats();
+    }
+
+    public void send_getTabletStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId) throws org.apache.thrift.TException
+    {
+      getTabletStats_args args = new getTabletStats_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setTableId(tableId);
+      sendBase("getTabletStats", args);
+    }
+
+    public java.util.List<TabletStats> recv_getTabletStats() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getTabletStats_result result = new getTabletStats_result();
+      receiveBase(result, "getTabletStats");
+      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, "getTabletStats failed: unknown result");
+    }
+
+    @Override
+    public TabletStats getHistoricalStats(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_getHistoricalStats(tinfo, credentials);
+      return recv_getHistoricalStats();
+    }
+
+    public void send_getHistoricalStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getHistoricalStats_args args = new getHistoricalStats_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getHistoricalStats", args);
+    }
+
+    public TabletStats recv_getHistoricalStats() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getHistoricalStats_result result = new getHistoricalStats_result();
+      receiveBase(result, "getHistoricalStats");
+      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, "getHistoricalStats failed: unknown result");
+    }
+
+    @Override
+    public void halt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_halt(tinfo, credentials, lock);
+      recv_halt();
+    }
+
+    public void send_halt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException
+    {
+      halt_args args = new halt_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      sendBase("halt", args);
+    }
+
+    public void recv_halt() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      halt_result result = new halt_result();
+      receiveBase(result, "halt");
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      return;
+    }
+
+    @Override
+    public void fastHalt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException
+    {
+      send_fastHalt(tinfo, credentials, lock);
+    }
+
+    public void send_fastHalt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock) throws org.apache.thrift.TException
+    {
+      fastHalt_args args = new fastHalt_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setLock(lock);
+      sendBaseOneway("fastHalt", args);
+    }
+
+    @Override
+    public java.util.List<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.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);
+      args.setCredentials(credentials);
+      sendBase("getActiveCompactions", args);
+    }
+
+    public java.util.List<ActiveCompaction> recv_getActiveCompactions() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getActiveCompactions_result result = new getActiveCompactions_result();
+      receiveBase(result, "getActiveCompactions");
+      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, "getActiveCompactions failed: unknown result");
+    }
+
+    @Override
+    public void removeLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException
+    {
+      send_removeLogs(tinfo, credentials, filenames);
+    }
+
+    public void send_removeLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames) throws org.apache.thrift.TException
+    {
+      removeLogs_args args = new removeLogs_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setFilenames(filenames);
+      sendBaseOneway("removeLogs", args);
+    }
+
+    @Override
+    public java.util.List<java.lang.String> getActiveLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      send_getActiveLogs(tinfo, credentials);
+      return recv_getActiveLogs();
+    }
+
+    public void send_getActiveLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getActiveLogs_args args = new getActiveLogs_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getActiveLogs", args);
+    }
+
+    public java.util.List<java.lang.String> recv_getActiveLogs() throws org.apache.thrift.TException
+    {
+      getActiveLogs_result result = new getActiveLogs_result();
+      receiveBase(result, "getActiveLogs");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveLogs failed: unknown result");
+    }
+
+    @Override
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_startGetSummaries(tinfo, credentials, request);
+      return recv_startGetSummaries();
+    }
+
+    public void send_startGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) throws org.apache.thrift.TException
+    {
+      startGetSummaries_args args = new startGetSummaries_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setRequest(request);
+      sendBase("startGetSummaries", args);
+    }
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_startGetSummaries() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    {
+      startGetSummaries_result result = new startGetSummaries_result();
+      receiveBase(result, "startGetSummaries");
+      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, "startGetSummaries failed: unknown result");
+    }
+
+    @Override
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesForPartition(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_startGetSummariesForPartition(tinfo, credentials, request, modulus, remainder);
+      return recv_startGetSummariesForPartition();
+    }
+
+    public void send_startGetSummariesForPartition(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder) throws org.apache.thrift.TException
+    {
+      startGetSummariesForPartition_args args = new startGetSummariesForPartition_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setRequest(request);
+      args.setModulus(modulus);
+      args.setRemainder(remainder);
+      sendBase("startGetSummariesForPartition", args);
+    }
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_startGetSummariesForPartition() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
+      receiveBase(result, "startGetSummariesForPartition");
+      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, "startGetSummariesForPartition failed: unknown result");
+    }
+
+    @Override
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries startGetSummariesFromFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_startGetSummariesFromFiles(tinfo, credentials, request, files);
+      return recv_startGetSummariesFromFiles();
+    }
+
+    public void send_startGetSummariesFromFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files) throws org.apache.thrift.TException
+    {
+      startGetSummariesFromFiles_args args = new startGetSummariesFromFiles_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setRequest(request);
+      args.setFiles(files);
+      sendBase("startGetSummariesFromFiles", args);
+    }
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_startGetSummariesFromFiles() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+      receiveBase(result, "startGetSummariesFromFiles");
+      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, "startGetSummariesFromFiles failed: unknown result");
+    }
+
+    @Override
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries contiuneGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessionId) throws NoSuchScanIDException, org.apache.thrift.TException
+    {
+      send_contiuneGetSummaries(tinfo, sessionId);
+      return recv_contiuneGetSummaries();
+    }
+
+    public void send_contiuneGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessionId) throws org.apache.thrift.TException
+    {
+      contiuneGetSummaries_args args = new contiuneGetSummaries_args();
+      args.setTinfo(tinfo);
+      args.setSessionId(sessionId);
+      sendBase("contiuneGetSummaries", args);
+    }
+
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries recv_contiuneGetSummaries() throws NoSuchScanIDException, org.apache.thrift.TException
+    {
+      contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+      receiveBase(result, "contiuneGetSummaries");
+      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, "contiuneGetSummaries failed: unknown result");
+    }
+
+    @Override
+    public java.util.List<TCompactionQueueSummary> getCompactionQueueInfo(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_getCompactionQueueInfo(tinfo, credentials);
+      return recv_getCompactionQueueInfo();
+    }
+
+    public void send_getCompactionQueueInfo(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    {
+      getCompactionQueueInfo_args args = new getCompactionQueueInfo_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      sendBase("getCompactionQueueInfo", args);
+    }
+
+    public java.util.List<TCompactionQueueSummary> recv_getCompactionQueueInfo() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
+      receiveBase(result, "getCompactionQueueInfo");
+      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, "getCompactionQueueInfo failed: unknown result");
+    }
+
+    @Override
+    public TExternalCompactionJob reserveCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      send_reserveCompactionJob(tinfo, credentials, queueName, priority, compactor, externalCompactionId);
+      return recv_reserveCompactionJob();
+    }
+
+    public void send_reserveCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId) throws org.apache.thrift.TException
+    {
+      reserveCompactionJob_args args = new reserveCompactionJob_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setQueueName(queueName);
+      args.setPriority(priority);
+      args.setCompactor(compactor);
+      args.setExternalCompactionId(externalCompactionId);
+      sendBase("reserveCompactionJob", args);
+    }
+
+    public TExternalCompactionJob recv_reserveCompactionJob() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    {
+      reserveCompactionJob_result result = new reserveCompactionJob_result();
+      receiveBase(result, "reserveCompactionJob");
+      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, "reserveCompactionJob failed: unknown result");
+    }
+
+    @Override
+    public void compactionJobFinished(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, long fileSize, long entries) throws org.apache.thrift.TException
+    {
+      send_compactionJobFinished(tinfo, credentials, externalCompactionId, extent, fileSize, entries);
+    }
+
+    public void send_compactionJobFinished(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, long fileSize, long entries) throws org.apache.thrift.TException
+    {
+      compactionJobFinished_args args = new compactionJobFinished_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExternalCompactionId(externalCompactionId);
+      args.setExtent(extent);
+      args.setFileSize(fileSize);
+      args.setEntries(entries);
+      sendBaseOneway("compactionJobFinished", args);
+    }
+
+    @Override
+    public void compactionJobFailed(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_compactionJobFailed(tinfo, credentials, externalCompactionId, extent);
+    }
+
+    public void send_compactionJobFailed(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
+    {
+      compactionJobFailed_args args = new compactionJobFailed_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setExternalCompactionId(externalCompactionId);
+      args.setExtent(extent);
+      sendBaseOneway("compactionJobFailed", 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 flush(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      flush_call method_call = new flush_call(tinfo, credentials, lock, tableId, startRow, endRow, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class flush_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 java.lang.String tableId;
+      private java.nio.ByteBuffer startRow;
+      private java.nio.ByteBuffer endRow;
+      public flush_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, 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.tableId = tableId;
+        this.startRow = startRow;
+        this.endRow = endRow;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("flush", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        flush_args args = new flush_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        args.setTableId(tableId);
+        args.setStartRow(startRow);
+        args.setEndRow(endRow);
+        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 compact(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      compact_call method_call = new compact_call(tinfo, credentials, lock, tableId, startRow, endRow, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compact_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 java.lang.String tableId;
+      private java.nio.ByteBuffer startRow;
+      private java.nio.ByteBuffer endRow;
+      public compact_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, java.lang.String tableId, java.nio.ByteBuffer startRow, java.nio.ByteBuffer endRow, 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.tableId = tableId;
+        this.startRow = startRow;
+        this.endRow = endRow;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compact", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        compact_args args = new compact_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        args.setTableId(tableId);
+        args.setStartRow(startRow);
+        args.setEndRow(endRow);
+        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 getTabletServerStatus(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.manager.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTabletServerStatus_call method_call = new getTabletServerStatus_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getTabletServerStatus_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.manager.thrift.TabletServerStatus> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getTabletServerStatus_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.manager.thrift.TabletServerStatus> 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;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTabletServerStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTabletServerStatus_args args = new getTabletServerStatus_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.manager.thrift.TabletServerStatus 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_getTabletServerStatus();
+      }
+    }
+
+    @Override
+    public void getTabletStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTabletStats_call method_call = new getTabletStats_call(tinfo, credentials, tableId, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getTabletStats_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<TabletStats>> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String tableId;
+      public getTabletStats_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String tableId, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> 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.tableId = tableId;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTabletStats", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTabletStats_args args = new getTabletStats_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setTableId(tableId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.util.List<TabletStats> 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_getTabletStats();
+      }
+    }
+
+    @Override
+    public void getHistoricalStats(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getHistoricalStats_call method_call = new getHistoricalStats_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getHistoricalStats_call extends org.apache.thrift.async.TAsyncMethodCall<TabletStats> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getHistoricalStats_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<TabletStats> 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;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getHistoricalStats", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getHistoricalStats_args args = new getHistoricalStats_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public TabletStats 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_getHistoricalStats();
+      }
+    }
+
+    @Override
+    public void halt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      halt_call method_call = new halt_call(tinfo, credentials, lock, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class halt_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;
+      public halt_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, 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.lock = lock;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("halt", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        halt_args args = new halt_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public Void 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);
+        (new Client(prot)).recv_halt();
+        return null;
+      }
+    }
+
+    @Override
+    public void fastHalt(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      fastHalt_call method_call = new fastHalt_call(tinfo, credentials, lock, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class fastHalt_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;
+      public fastHalt_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String lock, 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;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fastHalt", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        fastHalt_args args = new fastHalt_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setLock(lock);
+        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 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<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;
+      ___manager.call(method_call);
+    }
+
+    public static class getActiveCompactions_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<ActiveCompaction>> {
+      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.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<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;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveCompactions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getActiveCompactions_args args = new getActiveCompactions_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.util.List<ActiveCompaction> 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_getActiveCompactions();
+      }
+    }
+
+    @Override
+    public void removeLogs(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      removeLogs_call method_call = new removeLogs_call(tinfo, credentials, filenames, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class removeLogs_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.util.List<java.lang.String> filenames;
+      public removeLogs_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.util.List<java.lang.String> filenames, 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.filenames = filenames;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("removeLogs", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        removeLogs_args args = new removeLogs_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setFilenames(filenames);
+        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 getActiveLogs(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();
+      getActiveLogs_call method_call = new getActiveLogs_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getActiveLogs_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<java.lang.String>> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getActiveLogs_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;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getActiveLogs", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getActiveLogs_args args = new getActiveLogs_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.util.List<java.lang.String> getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getActiveLogs();
+      }
+    }
+
+    @Override
+    public void startGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startGetSummaries_call method_call = new startGetSummaries_call(tinfo, credentials, request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startGetSummaries_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      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.TSummaryRequest request;
+      public startGetSummaries_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.request = request;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummaries", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startGetSummaries_args args = new startGetSummaries_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.dataImpl.thrift.TSummaries getResult() throws org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException, org.apache.accumulo.core.clientImpl.thrift.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_startGetSummaries();
+      }
+    }
+
+    @Override
+    public void startGetSummariesForPartition(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startGetSummariesForPartition_call method_call = new startGetSummariesForPartition_call(tinfo, credentials, request, modulus, remainder, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startGetSummariesForPartition_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      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.TSummaryRequest request;
+      private int modulus;
+      private int remainder;
+      public startGetSummariesForPartition_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, int modulus, int remainder, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.request = request;
+        this.modulus = modulus;
+        this.remainder = remainder;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummariesForPartition", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startGetSummariesForPartition_args args = new startGetSummariesForPartition_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setRequest(request);
+        args.setModulus(modulus);
+        args.setRemainder(remainder);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.dataImpl.thrift.TSummaries 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_startGetSummariesForPartition();
+      }
+    }
+
+    @Override
+    public void startGetSummariesFromFiles(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      startGetSummariesFromFiles_call method_call = new startGetSummariesFromFiles_call(tinfo, credentials, request, files, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class startGetSummariesFromFiles_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      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.TSummaryRequest request;
+      private java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files;
+      public startGetSummariesFromFiles_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request, java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.request = request;
+        this.files = files;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("startGetSummariesFromFiles", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        startGetSummariesFromFiles_args args = new startGetSummariesFromFiles_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setRequest(request);
+        args.setFiles(files);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.dataImpl.thrift.TSummaries 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_startGetSummariesFromFiles();
+      }
+    }
+
+    @Override
+    public void contiuneGetSummaries(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      contiuneGetSummaries_call method_call = new contiuneGetSummaries_call(tinfo, sessionId, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class contiuneGetSummaries_call extends org.apache.thrift.async.TAsyncMethodCall<org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private long sessionId;
+      public contiuneGetSummaries_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, long sessionId, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.sessionId = sessionId;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("contiuneGetSummaries", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        contiuneGetSummaries_args args = new contiuneGetSummaries_args();
+        args.setTinfo(tinfo);
+        args.setSessionId(sessionId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public org.apache.accumulo.core.dataImpl.thrift.TSummaries getResult() throws 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_contiuneGetSummaries();
+      }
+    }
+
+    @Override
+    public void getCompactionQueueInfo(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getCompactionQueueInfo_call method_call = new getCompactionQueueInfo_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getCompactionQueueInfo_call extends org.apache.thrift.async.TAsyncMethodCall<java.util.List<TCompactionQueueSummary>> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      public getCompactionQueueInfo_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<TCompactionQueueSummary>> 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;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getCompactionQueueInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getCompactionQueueInfo_args args = new getCompactionQueueInfo_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public java.util.List<TCompactionQueueSummary> 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_getCompactionQueueInfo();
+      }
+    }
+
+    @Override
+    public void reserveCompactionJob(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      reserveCompactionJob_call method_call = new reserveCompactionJob_call(tinfo, credentials, queueName, priority, compactor, externalCompactionId, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class reserveCompactionJob_call extends org.apache.thrift.async.TAsyncMethodCall<TExternalCompactionJob> {
+      private org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials;
+      private java.lang.String queueName;
+      private long priority;
+      private java.lang.String compactor;
+      private java.lang.String externalCompactionId;
+      public reserveCompactionJob_call(org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo, org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials, java.lang.String queueName, long priority, java.lang.String compactor, java.lang.String externalCompactionId, org.apache.thrift.async.AsyncMethodCallback<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;
+        this.queueName = queueName;
+        this.priority = priority;
+        this.compactor = compactor;
+        this.externalCompactionId = externalCompactionId;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("reserveCompactionJob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        reserveCompactionJob_args args = new reserveCompactionJob_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setQueueName(queueName);
+        args.setPriority(priority);
+        args.setCompactor(compactor);
+        args.setExternalCompactionId(externalCompactionId);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      @Override
+      public TExternalCompactionJob 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_reserveCompactionJob();
+      }
+    }
+
+    @Override
+    public void compactionJobFinished(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, long fileSize, long entries, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      compactionJobFinished_call method_call = new compactionJobFinished_call(tinfo, credentials, externalCompactionId, extent, fileSize, entries, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compactionJobFinished_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 externalCompactionId;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      private long fileSize;
+      private long entries;
+      public compactionJobFinished_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, long fileSize, long entries, 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.externalCompactionId = externalCompactionId;
+        this.extent = extent;
+        this.fileSize = fileSize;
+        this.entries = entries;
+      }
+
+      @Override
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compactionJobFinished", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        compactionJobFinished_args args = new compactionJobFinished_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExternalCompactionId(externalCompactionId);
+        args.setExtent(extent);
+        args.setFileSize(fileSize);
+        args.setEntries(entries);
+        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 compactionJobFailed(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();
+      compactionJobFailed_call method_call = new compactionJobFailed_call(tinfo, credentials, externalCompactionId, extent, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class compactionJobFailed_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 externalCompactionId;
+      private org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent;
+      public compactionJobFailed_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, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.externalCompactionId = externalCompactionId;
+        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("compactionJobFailed", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        compactionJobFailed_args args = new compactionJobFailed_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setExternalCompactionId(externalCompactionId);
+        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("flush", new flush());
+      processMap.put("compact", new compact());
+      processMap.put("getTabletServerStatus", new getTabletServerStatus());
+      processMap.put("getTabletStats", new getTabletStats());
+      processMap.put("getHistoricalStats", new getHistoricalStats());
+      processMap.put("halt", new halt());
+      processMap.put("fastHalt", new fastHalt());
+      processMap.put("getActiveCompactions", new getActiveCompactions());
+      processMap.put("removeLogs", new removeLogs());
+      processMap.put("getActiveLogs", new getActiveLogs());
+      processMap.put("startGetSummaries", new startGetSummaries());
+      processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
+      processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
+      processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
+      processMap.put("getCompactionQueueInfo", new getCompactionQueueInfo());
+      processMap.put("reserveCompactionJob", new reserveCompactionJob());
+      processMap.put("compactionJobFinished", new compactionJobFinished());
+      processMap.put("compactionJobFailed", new compactionJobFailed());
+      return processMap;
+    }
+
+    public static class flush<I extends Iface> extends org.apache.thrift.ProcessFunction<I, flush_args> {
+      public flush() {
+        super("flush");
+      }
+
+      @Override
+      public flush_args getEmptyArgsInstance() {
+        return new flush_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, flush_args args) throws org.apache.thrift.TException {
+        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow);
+        return null;
+      }
+    }
+
+    public static class compact<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compact_args> {
+      public compact() {
+        super("compact");
+      }
+
+      @Override
+      public compact_args getEmptyArgsInstance() {
+        return new compact_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, compact_args args) throws org.apache.thrift.TException {
+        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow);
+        return null;
+      }
+    }
+
+    public static class getTabletServerStatus<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTabletServerStatus_args> {
+      public getTabletServerStatus() {
+        super("getTabletServerStatus");
+      }
+
+      @Override
+      public getTabletServerStatus_args getEmptyArgsInstance() {
+        return new getTabletServerStatus_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public getTabletServerStatus_result getResult(I iface, getTabletServerStatus_args args) throws org.apache.thrift.TException {
+        getTabletServerStatus_result result = new getTabletServerStatus_result();
+        try {
+          result.success = iface.getTabletServerStatus(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class getTabletStats<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTabletStats_args> {
+      public getTabletStats() {
+        super("getTabletStats");
+      }
+
+      @Override
+      public getTabletStats_args getEmptyArgsInstance() {
+        return new getTabletStats_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public getTabletStats_result getResult(I iface, getTabletStats_args args) throws org.apache.thrift.TException {
+        getTabletStats_result result = new getTabletStats_result();
+        try {
+          result.success = iface.getTabletStats(args.tinfo, args.credentials, args.tableId);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class getHistoricalStats<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getHistoricalStats_args> {
+      public getHistoricalStats() {
+        super("getHistoricalStats");
+      }
+
+      @Override
+      public getHistoricalStats_args getEmptyArgsInstance() {
+        return new getHistoricalStats_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public getHistoricalStats_result getResult(I iface, getHistoricalStats_args args) throws org.apache.thrift.TException {
+        getHistoricalStats_result result = new getHistoricalStats_result();
+        try {
+          result.success = iface.getHistoricalStats(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class halt<I extends Iface> extends org.apache.thrift.ProcessFunction<I, halt_args> {
+      public halt() {
+        super("halt");
+      }
+
+      @Override
+      public halt_args getEmptyArgsInstance() {
+        return new halt_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public halt_result getResult(I iface, halt_args args) throws org.apache.thrift.TException {
+        halt_result result = new halt_result();
+        try {
+          iface.halt(args.tinfo, args.credentials, args.lock);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class fastHalt<I extends Iface> extends org.apache.thrift.ProcessFunction<I, fastHalt_args> {
+      public fastHalt() {
+        super("fastHalt");
+      }
+
+      @Override
+      public fastHalt_args getEmptyArgsInstance() {
+        return new fastHalt_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, fastHalt_args args) throws org.apache.thrift.TException {
+        iface.fastHalt(args.tinfo, args.credentials, args.lock);
+        return null;
+      }
+    }
+
+    public static class getActiveCompactions<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveCompactions_args> {
+      public getActiveCompactions() {
+        super("getActiveCompactions");
+      }
+
+      @Override
+      public getActiveCompactions_args getEmptyArgsInstance() {
+        return new getActiveCompactions_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public getActiveCompactions_result getResult(I iface, getActiveCompactions_args args) throws org.apache.thrift.TException {
+        getActiveCompactions_result result = new getActiveCompactions_result();
+        try {
+          result.success = iface.getActiveCompactions(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class removeLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, removeLogs_args> {
+      public removeLogs() {
+        super("removeLogs");
+      }
+
+      @Override
+      public removeLogs_args getEmptyArgsInstance() {
+        return new removeLogs_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, removeLogs_args args) throws org.apache.thrift.TException {
+        iface.removeLogs(args.tinfo, args.credentials, args.filenames);
+        return null;
+      }
+    }
+
+    public static class getActiveLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveLogs_args> {
+      public getActiveLogs() {
+        super("getActiveLogs");
+      }
+
+      @Override
+      public getActiveLogs_args getEmptyArgsInstance() {
+        return new getActiveLogs_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public getActiveLogs_result getResult(I iface, getActiveLogs_args args) throws org.apache.thrift.TException {
+        getActiveLogs_result result = new getActiveLogs_result();
+        result.success = iface.getActiveLogs(args.tinfo, args.credentials);
+        return result;
+      }
+    }
+
+    public static class startGetSummaries<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummaries_args> {
+      public startGetSummaries() {
+        super("startGetSummaries");
+      }
+
+      @Override
+      public startGetSummaries_args getEmptyArgsInstance() {
+        return new startGetSummaries_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public startGetSummaries_result getResult(I iface, startGetSummaries_args args) throws org.apache.thrift.TException {
+        startGetSummaries_result result = new startGetSummaries_result();
+        try {
+          result.success = iface.startGetSummaries(args.tinfo, args.credentials, args.request);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class startGetSummariesForPartition<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummariesForPartition_args> {
+      public startGetSummariesForPartition() {
+        super("startGetSummariesForPartition");
+      }
+
+      @Override
+      public startGetSummariesForPartition_args getEmptyArgsInstance() {
+        return new startGetSummariesForPartition_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public startGetSummariesForPartition_result getResult(I iface, startGetSummariesForPartition_args args) throws org.apache.thrift.TException {
+        startGetSummariesForPartition_result result = new startGetSummariesForPartition_result();
+        try {
+          result.success = iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class startGetSummariesFromFiles<I extends Iface> extends org.apache.thrift.ProcessFunction<I, startGetSummariesFromFiles_args> {
+      public startGetSummariesFromFiles() {
+        super("startGetSummariesFromFiles");
+      }
+
+      @Override
+      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
+        return new startGetSummariesFromFiles_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public startGetSummariesFromFiles_result getResult(I iface, startGetSummariesFromFiles_args args) throws org.apache.thrift.TException {
+        startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_result();
+        try {
+          result.success = iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class contiuneGetSummaries<I extends Iface> extends org.apache.thrift.ProcessFunction<I, contiuneGetSummaries_args> {
+      public contiuneGetSummaries() {
+        super("contiuneGetSummaries");
+      }
+
+      @Override
+      public contiuneGetSummaries_args getEmptyArgsInstance() {
+        return new contiuneGetSummaries_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public contiuneGetSummaries_result getResult(I iface, contiuneGetSummaries_args args) throws org.apache.thrift.TException {
+        contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+        try {
+          result.success = iface.contiuneGetSummaries(args.tinfo, args.sessionId);
+        } catch (NoSuchScanIDException nssi) {
+          result.nssi = nssi;
+        }
+        return result;
+      }
+    }
+
+    public static class getCompactionQueueInfo<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getCompactionQueueInfo_args> {
+      public getCompactionQueueInfo() {
+        super("getCompactionQueueInfo");
+      }
+
+      @Override
+      public getCompactionQueueInfo_args getEmptyArgsInstance() {
+        return new getCompactionQueueInfo_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public getCompactionQueueInfo_result getResult(I iface, getCompactionQueueInfo_args args) throws org.apache.thrift.TException {
+        getCompactionQueueInfo_result result = new getCompactionQueueInfo_result();
+        try {
+          result.success = iface.getCompactionQueueInfo(args.tinfo, args.credentials);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class reserveCompactionJob<I extends Iface> extends org.apache.thrift.ProcessFunction<I, reserveCompactionJob_args> {
+      public reserveCompactionJob() {
+        super("reserveCompactionJob");
+      }
+
+      @Override
+      public reserveCompactionJob_args getEmptyArgsInstance() {
+        return new reserveCompactionJob_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return false;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public reserveCompactionJob_result getResult(I iface, reserveCompactionJob_args args) throws org.apache.thrift.TException {
+        reserveCompactionJob_result result = new reserveCompactionJob_result();
+        try {
+          result.success = iface.reserveCompactionJob(args.tinfo, args.credentials, args.queueName, args.priority, args.compactor, args.externalCompactionId);
+        } catch (org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec) {
+          result.sec = sec;
+        }
+        return result;
+      }
+    }
+
+    public static class compactionJobFinished<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionJobFinished_args> {
+      public compactionJobFinished() {
+        super("compactionJobFinished");
+      }
+
+      @Override
+      public compactionJobFinished_args getEmptyArgsInstance() {
+        return new compactionJobFinished_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, compactionJobFinished_args args) throws org.apache.thrift.TException {
+        iface.compactionJobFinished(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.fileSize, args.entries);
+        return null;
+      }
+    }
+
+    public static class compactionJobFailed<I extends Iface> extends org.apache.thrift.ProcessFunction<I, compactionJobFailed_args> {
+      public compactionJobFailed() {
+        super("compactionJobFailed");
+      }
+
+      @Override
+      public compactionJobFailed_args getEmptyArgsInstance() {
+        return new compactionJobFailed_args();
+      }
+
+      @Override
+      protected boolean isOneway() {
+        return true;
+      }
+
+      @Override
+      protected boolean rethrowUnhandledExceptions() {
+        return false;
+      }
+
+      @Override
+      public org.apache.thrift.TBase getResult(I iface, compactionJobFailed_args args) throws org.apache.thrift.TException {
+        iface.compactionJobFailed(args.tinfo, args.credentials, args.externalCompactionId, 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("flush", new flush());
+      processMap.put("compact", new compact());
+      processMap.put("getTabletServerStatus", new getTabletServerStatus());
+      processMap.put("getTabletStats", new getTabletStats());
+      processMap.put("getHistoricalStats", new getHistoricalStats());
+      processMap.put("halt", new halt());
+      processMap.put("fastHalt", new fastHalt());
+      processMap.put("getActiveCompactions", new getActiveCompactions());
+      processMap.put("removeLogs", new removeLogs());
+      processMap.put("getActiveLogs", new getActiveLogs());
+      processMap.put("startGetSummaries", new startGetSummaries());
+      processMap.put("startGetSummariesForPartition", new startGetSummariesForPartition());
+      processMap.put("startGetSummariesFromFiles", new startGetSummariesFromFiles());
+      processMap.put("contiuneGetSummaries", new contiuneGetSummaries());
+      processMap.put("getCompactionQueueInfo", new getCompactionQueueInfo());
+      processMap.put("reserveCompactionJob", new reserveCompactionJob());
+      processMap.put("compactionJobFinished", new compactionJobFinished());
+      processMap.put("compactionJobFailed", new compactionJobFailed());
+      return processMap;
+    }
+
+    public static class flush<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, flush_args, Void> {
+      public flush() {
+        super("flush");
+      }
+
+      @Override
+      public flush_args getEmptyArgsInstance() {
+        return new flush_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, flush_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.flush(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      }
+    }
+
+    public static class compact<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compact_args, Void> {
+      public compact() {
+        super("compact");
+      }
+
+      @Override
+      public compact_args getEmptyArgsInstance() {
+        return new compact_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, compact_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compact(args.tinfo, args.credentials, args.lock, args.tableId, args.startRow, args.endRow,resultHandler);
+      }
+    }
+
+    public static class getTabletServerStatus<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletServerStatus_args, org.apache.accumulo.core.manager.thrift.TabletServerStatus> {
+      public getTabletServerStatus() {
+        super("getTabletServerStatus");
+      }
+
+      @Override
+      public getTabletServerStatus_args getEmptyArgsInstance() {
+        return new getTabletServerStatus_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.manager.thrift.TabletServerStatus> 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.manager.thrift.TabletServerStatus>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.manager.thrift.TabletServerStatus o) {
+            getTabletServerStatus_result result = new getTabletServerStatus_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;
+            getTabletServerStatus_result result = new getTabletServerStatus_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, getTabletServerStatus_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.manager.thrift.TabletServerStatus> resultHandler) throws org.apache.thrift.TException {
+        iface.getTabletServerStatus(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class getTabletStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTabletStats_args, java.util.List<TabletStats>> {
+      public getTabletStats() {
+        super("getTabletStats");
+      }
+
+      @Override
+      public getTabletStats_args getEmptyArgsInstance() {
+        return new getTabletStats_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> 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<TabletStats>>() { 
+          @Override
+          public void onComplete(java.util.List<TabletStats> o) {
+            getTabletStats_result result = new getTabletStats_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;
+            getTabletStats_result result = new getTabletStats_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, getTabletStats_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TabletStats>> resultHandler) throws org.apache.thrift.TException {
+        iface.getTabletStats(args.tinfo, args.credentials, args.tableId,resultHandler);
+      }
+    }
+
+    public static class getHistoricalStats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getHistoricalStats_args, TabletStats> {
+      public getHistoricalStats() {
+        super("getHistoricalStats");
+      }
+
+      @Override
+      public getHistoricalStats_args getEmptyArgsInstance() {
+        return new getHistoricalStats_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<TabletStats> 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<TabletStats>() { 
+          @Override
+          public void onComplete(TabletStats o) {
+            getHistoricalStats_result result = new getHistoricalStats_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;
+            getHistoricalStats_result result = new getHistoricalStats_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, getHistoricalStats_args args, org.apache.thrift.async.AsyncMethodCallback<TabletStats> resultHandler) throws org.apache.thrift.TException {
+        iface.getHistoricalStats(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class halt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, halt_args, Void> {
+      public halt() {
+        super("halt");
+      }
+
+      @Override
+      public halt_args getEmptyArgsInstance() {
+        return new halt_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) {
+            halt_result result = new halt_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;
+            halt_result result = new halt_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, halt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.halt(args.tinfo, args.credentials, args.lock,resultHandler);
+      }
+    }
+
+    public static class fastHalt<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, fastHalt_args, Void> {
+      public fastHalt() {
+        super("fastHalt");
+      }
+
+      @Override
+      public fastHalt_args getEmptyArgsInstance() {
+        return new fastHalt_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, fastHalt_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.fastHalt(args.tinfo, args.credentials, args.lock,resultHandler);
+      }
+    }
+
+    public static class getActiveCompactions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveCompactions_args, java.util.List<ActiveCompaction>> {
+      public getActiveCompactions() {
+        super("getActiveCompactions");
+      }
+
+      @Override
+      public getActiveCompactions_args getEmptyArgsInstance() {
+        return new getActiveCompactions_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> 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<ActiveCompaction>>() { 
+          @Override
+          public void onComplete(java.util.List<ActiveCompaction> o) {
+            getActiveCompactions_result result = new getActiveCompactions_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;
+            getActiveCompactions_result result = new getActiveCompactions_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, getActiveCompactions_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<ActiveCompaction>> resultHandler) throws org.apache.thrift.TException {
+        iface.getActiveCompactions(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class removeLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, removeLogs_args, Void> {
+      public removeLogs() {
+        super("removeLogs");
+      }
+
+      @Override
+      public removeLogs_args getEmptyArgsInstance() {
+        return new removeLogs_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, removeLogs_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.removeLogs(args.tinfo, args.credentials, args.filenames,resultHandler);
+      }
+    }
+
+    public static class getActiveLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveLogs_args, java.util.List<java.lang.String>> {
+      public getActiveLogs() {
+        super("getActiveLogs");
+      }
+
+      @Override
+      public getActiveLogs_args getEmptyArgsInstance() {
+        return new getActiveLogs_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) {
+            getActiveLogs_result result = new getActiveLogs_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;
+            getActiveLogs_result result = new getActiveLogs_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, getActiveLogs_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<java.lang.String>> resultHandler) throws org.apache.thrift.TException {
+        iface.getActiveLogs(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class startGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public startGetSummaries() {
+        super("startGetSummaries");
+      }
+
+      @Override
+      public startGetSummaries_args getEmptyArgsInstance() {
+        return new startGetSummaries_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            startGetSummaries_result result = new startGetSummaries_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;
+            startGetSummaries_result result = new startGetSummaries_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.clientImpl.thrift.ThriftTableOperationException) {
+              result.tope = (org.apache.accumulo.core.clientImpl.thrift.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, startGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummaries(args.tinfo, args.credentials, args.request,resultHandler);
+      }
+    }
+
+    public static class startGetSummariesForPartition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesForPartition_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public startGetSummariesForPartition() {
+        super("startGetSummariesForPartition");
+      }
+
+      @Override
+      public startGetSummariesForPartition_args getEmptyArgsInstance() {
+        return new startGetSummariesForPartition_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            startGetSummariesForPartition_result result = new startGetSummariesForPartition_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;
+            startGetSummariesForPartition_result result = new startGetSummariesForPartition_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, startGetSummariesForPartition_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummariesForPartition(args.tinfo, args.credentials, args.request, args.modulus, args.remainder,resultHandler);
+      }
+    }
+
+    public static class startGetSummariesFromFiles<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, startGetSummariesFromFiles_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public startGetSummariesFromFiles() {
+        super("startGetSummariesFromFiles");
+      }
+
+      @Override
+      public startGetSummariesFromFiles_args getEmptyArgsInstance() {
+        return new startGetSummariesFromFiles_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_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;
+            startGetSummariesFromFiles_result result = new startGetSummariesFromFiles_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, startGetSummariesFromFiles_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.startGetSummariesFromFiles(args.tinfo, args.credentials, args.request, args.files,resultHandler);
+      }
+    }
+
+    public static class contiuneGetSummaries<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, contiuneGetSummaries_args, org.apache.accumulo.core.dataImpl.thrift.TSummaries> {
+      public contiuneGetSummaries() {
+        super("contiuneGetSummaries");
+      }
+
+      @Override
+      public contiuneGetSummaries_args getEmptyArgsInstance() {
+        return new contiuneGetSummaries_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> 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.TSummaries>() { 
+          @Override
+          public void onComplete(org.apache.accumulo.core.dataImpl.thrift.TSummaries o) {
+            contiuneGetSummaries_result result = new contiuneGetSummaries_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;
+            contiuneGetSummaries_result result = new contiuneGetSummaries_result();
+            if (e instanceof NoSuchScanIDException) {
+              result.nssi = (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, contiuneGetSummaries_args args, org.apache.thrift.async.AsyncMethodCallback<org.apache.accumulo.core.dataImpl.thrift.TSummaries> resultHandler) throws org.apache.thrift.TException {
+        iface.contiuneGetSummaries(args.tinfo, args.sessionId,resultHandler);
+      }
+    }
+
+    public static class getCompactionQueueInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getCompactionQueueInfo_args, java.util.List<TCompactionQueueSummary>> {
+      public getCompactionQueueInfo() {
+        super("getCompactionQueueInfo");
+      }
+
+      @Override
+      public getCompactionQueueInfo_args getEmptyArgsInstance() {
+        return new getCompactionQueueInfo_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> 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<TCompactionQueueSummary>>() { 
+          @Override
+          public void onComplete(java.util.List<TCompactionQueueSummary> o) {
+            getCompactionQueueInfo_result result = new getCompactionQueueInfo_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;
+            getCompactionQueueInfo_result result = new getCompactionQueueInfo_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, getCompactionQueueInfo_args args, org.apache.thrift.async.AsyncMethodCallback<java.util.List<TCompactionQueueSummary>> resultHandler) throws org.apache.thrift.TException {
+        iface.getCompactionQueueInfo(args.tinfo, args.credentials,resultHandler);
+      }
+    }
+
+    public static class reserveCompactionJob<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, reserveCompactionJob_args, TExternalCompactionJob> {
+      public reserveCompactionJob() {
+        super("reserveCompactionJob");
+      }
+
+      @Override
+      public reserveCompactionJob_args getEmptyArgsInstance() {
+        return new reserveCompactionJob_args();
+      }
+
+      @Override
+      public org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> 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<TExternalCompactionJob>() { 
+          @Override
+          public void onComplete(TExternalCompactionJob o) {
+            reserveCompactionJob_result result = new reserveCompactionJob_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;
+            reserveCompactionJob_result result = new reserveCompactionJob_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, reserveCompactionJob_args args, org.apache.thrift.async.AsyncMethodCallback<TExternalCompactionJob> resultHandler) throws org.apache.thrift.TException {
+        iface.reserveCompactionJob(args.tinfo, args.credentials, args.queueName, args.priority, args.compactor, args.externalCompactionId,resultHandler);
+      }
+    }
+
+    public static class compactionJobFinished<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFinished_args, Void> {
+      public compactionJobFinished() {
+        super("compactionJobFinished");
+      }
+
+      @Override
+      public compactionJobFinished_args getEmptyArgsInstance() {
+        return new compactionJobFinished_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, compactionJobFinished_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionJobFinished(args.tinfo, args.credentials, args.externalCompactionId, args.extent, args.fileSize, args.entries,resultHandler);
+      }
+    }
+
+    public static class compactionJobFailed<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, compactionJobFailed_args, Void> {
+      public compactionJobFailed() {
+        super("compactionJobFailed");
+      }
+
+      @Override
+      public compactionJobFailed_args getEmptyArgsInstance() {
+        return new compactionJobFailed_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, compactionJobFailed_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.compactionJobFailed(args.tinfo, args.credentials, args.externalCompactionId, args.extent,resultHandler);
+      }
+    }
+
+  }
+
+  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+  public static class flush_args implements org.apache.thrift.TBase<flush_args, flush_args._Fields>, java.io.Serializable, Cloneable, Comparable<flush_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("flush_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 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 TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)5);
+    private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (short)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new flush_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new flush_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 java.lang.String tableId; // required
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow; // required
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow; // 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"),
+      LOCK((short)3, "lock"),
+      TABLE_ID((short)2, "tableId"),
+      START_ROW((short)5, "startRow"),
+      END_ROW((short)6, "endRow");
+
+      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 3: // LOCK
+            return LOCK;
+          case 2: // TABLE_ID
+            return TABLE_ID;
+          case 5: // START_ROW
+            return START_ROW;
+          case 6: // END_ROW
+            return END_ROW;
+          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.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.START_ROW, new org.apache.thrift.meta_data.FieldMetaData("startRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", 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(flush_args.class, metaDataMap);
+    }
+
+    public flush_args() {
+    }
+
+    public flush_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock,
+      java.lang.String tableId,
+      java.nio.ByteBuffer startRow,
+      java.nio.ByteBuffer endRow)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+      this.tableId = tableId;
+      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
+      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public flush_args(flush_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.isSetTableId()) {
+        this.tableId = other.tableId;
+      }
+      if (other.isSetStartRow()) {
+        this.startRow = org.apache.thrift.TBaseHelper.copyBinary(other.startRow);
+      }
+      if (other.isSetEndRow()) {
+        this.endRow = org.apache.thrift.TBaseHelper.copyBinary(other.endRow);
+      }
+    }
+
+    @Override
+    public flush_args deepCopy() {
+      return new flush_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+      this.tableId = null;
+      this.startRow = null;
+      this.endRow = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public flush_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 flush_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 flush_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 java.lang.String getTableId() {
+      return this.tableId;
+    }
+
+    public flush_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 byte[] getStartRow() {
+      setStartRow(org.apache.thrift.TBaseHelper.rightSize(startRow));
+      return startRow == null ? null : startRow.array();
+    }
+
+    public java.nio.ByteBuffer bufferForStartRow() {
+      return org.apache.thrift.TBaseHelper.copyBinary(startRow);
+    }
+
+    public flush_args setStartRow(byte[] startRow) {
+      this.startRow = startRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(startRow.clone());
+      return this;
+    }
+
+    public flush_args setStartRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow) {
+      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
+      return this;
+    }
+
+    public void unsetStartRow() {
+      this.startRow = null;
+    }
+
+    /** Returns true if field startRow is set (has been assigned a value) and false otherwise */
+    public boolean isSetStartRow() {
+      return this.startRow != null;
+    }
+
+    public void setStartRowIsSet(boolean value) {
+      if (!value) {
+        this.startRow = null;
+      }
+    }
+
+    public byte[] getEndRow() {
+      setEndRow(org.apache.thrift.TBaseHelper.rightSize(endRow));
+      return endRow == null ? null : endRow.array();
+    }
+
+    public java.nio.ByteBuffer bufferForEndRow() {
+      return org.apache.thrift.TBaseHelper.copyBinary(endRow);
+    }
+
+    public flush_args setEndRow(byte[] endRow) {
+      this.endRow = endRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(endRow.clone());
+      return this;
+    }
+
+    public flush_args setEndRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow) {
+      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
+      return this;
+    }
+
+    public void unsetEndRow() {
+      this.endRow = null;
+    }
+
+    /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
+    public boolean isSetEndRow() {
+      return this.endRow != null;
+    }
+
+    public void setEndRowIsSet(boolean value) {
+      if (!value) {
+        this.endRow = 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 TABLE_ID:
+        if (value == null) {
+          unsetTableId();
+        } else {
+          setTableId((java.lang.String)value);
+        }
+        break;
+
+      case START_ROW:
+        if (value == null) {
+          unsetStartRow();
+        } else {
+          if (value instanceof byte[]) {
+            setStartRow((byte[])value);
+          } else {
+            setStartRow((java.nio.ByteBuffer)value);
+          }
+        }
+        break;
+
+      case END_ROW:
+        if (value == null) {
+          unsetEndRow();
+        } else {
+          if (value instanceof byte[]) {
+            setEndRow((byte[])value);
+          } else {
+            setEndRow((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 LOCK:
+        return getLock();
+
+      case TABLE_ID:
+        return getTableId();
+
+      case START_ROW:
+        return getStartRow();
+
+      case END_ROW:
+        return getEndRow();
+
+      }
+      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 TABLE_ID:
+        return isSetTableId();
+      case START_ROW:
+        return isSetStartRow();
+      case END_ROW:
+        return isSetEndRow();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof flush_args)
+        return this.equals((flush_args)that);
+      return false;
+    }
+
+    public boolean equals(flush_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_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_startRow = true && this.isSetStartRow();
+      boolean that_present_startRow = true && that.isSetStartRow();
+      if (this_present_startRow || that_present_startRow) {
+        if (!(this_present_startRow && that_present_startRow))
+          return false;
+        if (!this.startRow.equals(that.startRow))
+          return false;
+      }
+
+      boolean this_present_endRow = true && this.isSetEndRow();
+      boolean that_present_endRow = true && that.isSetEndRow();
+      if (this_present_endRow || that_present_endRow) {
+        if (!(this_present_endRow && that_present_endRow))
+          return false;
+        if (!this.endRow.equals(that.endRow))
+          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 + ((isSetTableId()) ? 131071 : 524287);
+      if (isSetTableId())
+        hashCode = hashCode * 8191 + tableId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetStartRow()) ? 131071 : 524287);
+      if (isSetStartRow())
+        hashCode = hashCode * 8191 + startRow.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
+      if (isSetEndRow())
+        hashCode = hashCode * 8191 + endRow.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(flush_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(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(isSetStartRow(), other.isSetStartRow());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetStartRow()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetEndRow(), other.isSetEndRow());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetEndRow()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
+        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("flush_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("tableId:");
+      if (this.tableId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("startRow:");
+      if (this.startRow == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.startRow, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("endRow:");
+      if (this.endRow == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.endRow, 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();
+      }
+    }
+
+    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 flush_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public flush_argsStandardScheme getScheme() {
+        return new flush_argsStandardScheme();
+      }
+    }
+
+    private static class flush_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<flush_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, flush_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 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 2: // 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: // START_ROW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.startRow = iprot.readBinary();
+                struct.setStartRowIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // END_ROW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.endRow = iprot.readBinary();
+                struct.setEndRowIsSet(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, flush_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.tableId != null) {
+          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+          oprot.writeString(struct.tableId);
+          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.startRow != null) {
+          oprot.writeFieldBegin(START_ROW_FIELD_DESC);
+          oprot.writeBinary(struct.startRow);
+          oprot.writeFieldEnd();
+        }
+        if (struct.endRow != null) {
+          oprot.writeFieldBegin(END_ROW_FIELD_DESC);
+          oprot.writeBinary(struct.endRow);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class flush_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public flush_argsTupleScheme getScheme() {
+        return new flush_argsTupleScheme();
+      }
+    }
+
+    private static class flush_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<flush_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, flush_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.isSetTableId()) {
+          optionals.set(3);
+        }
+        if (struct.isSetStartRow()) {
+          optionals.set(4);
+        }
+        if (struct.isSetEndRow()) {
+          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.isSetTableId()) {
+          oprot.writeString(struct.tableId);
+        }
+        if (struct.isSetStartRow()) {
+          oprot.writeBinary(struct.startRow);
+        }
+        if (struct.isSetEndRow()) {
+          oprot.writeBinary(struct.endRow);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, flush_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.tableId = iprot.readString();
+          struct.setTableIdIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.startRow = iprot.readBinary();
+          struct.setStartRowIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.endRow = iprot.readBinary();
+          struct.setEndRowIsSet(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 compact_args implements org.apache.thrift.TBase<compact_args, compact_args._Fields>, java.io.Serializable, Cloneable, Comparable<compact_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_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 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 START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)5);
+    private static final org.apache.thrift.protocol.TField END_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("endRow", org.apache.thrift.protocol.TType.STRING, (short)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compact_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compact_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 java.lang.String tableId; // required
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow; // required
+    public @org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow; // 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"),
+      TABLE_ID((short)4, "tableId"),
+      START_ROW((short)5, "startRow"),
+      END_ROW((short)6, "endRow");
+
+      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: // TABLE_ID
+            return TABLE_ID;
+          case 5: // START_ROW
+            return START_ROW;
+          case 6: // END_ROW
+            return END_ROW;
+          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.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.START_ROW, new org.apache.thrift.meta_data.FieldMetaData("startRow", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.END_ROW, new org.apache.thrift.meta_data.FieldMetaData("endRow", 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(compact_args.class, metaDataMap);
+    }
+
+    public compact_args() {
+    }
+
+    public compact_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock,
+      java.lang.String tableId,
+      java.nio.ByteBuffer startRow,
+      java.nio.ByteBuffer endRow)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+      this.tableId = tableId;
+      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
+      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compact_args(compact_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.isSetTableId()) {
+        this.tableId = other.tableId;
+      }
+      if (other.isSetStartRow()) {
+        this.startRow = org.apache.thrift.TBaseHelper.copyBinary(other.startRow);
+      }
+      if (other.isSetEndRow()) {
+        this.endRow = org.apache.thrift.TBaseHelper.copyBinary(other.endRow);
+      }
+    }
+
+    @Override
+    public compact_args deepCopy() {
+      return new compact_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+      this.tableId = null;
+      this.startRow = null;
+      this.endRow = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public compact_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 compact_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 compact_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 java.lang.String getTableId() {
+      return this.tableId;
+    }
+
+    public compact_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 byte[] getStartRow() {
+      setStartRow(org.apache.thrift.TBaseHelper.rightSize(startRow));
+      return startRow == null ? null : startRow.array();
+    }
+
+    public java.nio.ByteBuffer bufferForStartRow() {
+      return org.apache.thrift.TBaseHelper.copyBinary(startRow);
+    }
+
+    public compact_args setStartRow(byte[] startRow) {
+      this.startRow = startRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(startRow.clone());
+      return this;
+    }
+
+    public compact_args setStartRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer startRow) {
+      this.startRow = org.apache.thrift.TBaseHelper.copyBinary(startRow);
+      return this;
+    }
+
+    public void unsetStartRow() {
+      this.startRow = null;
+    }
+
+    /** Returns true if field startRow is set (has been assigned a value) and false otherwise */
+    public boolean isSetStartRow() {
+      return this.startRow != null;
+    }
+
+    public void setStartRowIsSet(boolean value) {
+      if (!value) {
+        this.startRow = null;
+      }
+    }
+
+    public byte[] getEndRow() {
+      setEndRow(org.apache.thrift.TBaseHelper.rightSize(endRow));
+      return endRow == null ? null : endRow.array();
+    }
+
+    public java.nio.ByteBuffer bufferForEndRow() {
+      return org.apache.thrift.TBaseHelper.copyBinary(endRow);
+    }
+
+    public compact_args setEndRow(byte[] endRow) {
+      this.endRow = endRow == null ? (java.nio.ByteBuffer)null     : java.nio.ByteBuffer.wrap(endRow.clone());
+      return this;
+    }
+
+    public compact_args setEndRow(@org.apache.thrift.annotation.Nullable java.nio.ByteBuffer endRow) {
+      this.endRow = org.apache.thrift.TBaseHelper.copyBinary(endRow);
+      return this;
+    }
+
+    public void unsetEndRow() {
+      this.endRow = null;
+    }
+
+    /** Returns true if field endRow is set (has been assigned a value) and false otherwise */
+    public boolean isSetEndRow() {
+      return this.endRow != null;
+    }
+
+    public void setEndRowIsSet(boolean value) {
+      if (!value) {
+        this.endRow = 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 TABLE_ID:
+        if (value == null) {
+          unsetTableId();
+        } else {
+          setTableId((java.lang.String)value);
+        }
+        break;
+
+      case START_ROW:
+        if (value == null) {
+          unsetStartRow();
+        } else {
+          if (value instanceof byte[]) {
+            setStartRow((byte[])value);
+          } else {
+            setStartRow((java.nio.ByteBuffer)value);
+          }
+        }
+        break;
+
+      case END_ROW:
+        if (value == null) {
+          unsetEndRow();
+        } else {
+          if (value instanceof byte[]) {
+            setEndRow((byte[])value);
+          } else {
+            setEndRow((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 LOCK:
+        return getLock();
+
+      case TABLE_ID:
+        return getTableId();
+
+      case START_ROW:
+        return getStartRow();
+
+      case END_ROW:
+        return getEndRow();
+
+      }
+      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 TABLE_ID:
+        return isSetTableId();
+      case START_ROW:
+        return isSetStartRow();
+      case END_ROW:
+        return isSetEndRow();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof compact_args)
+        return this.equals((compact_args)that);
+      return false;
+    }
+
+    public boolean equals(compact_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_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_startRow = true && this.isSetStartRow();
+      boolean that_present_startRow = true && that.isSetStartRow();
+      if (this_present_startRow || that_present_startRow) {
+        if (!(this_present_startRow && that_present_startRow))
+          return false;
+        if (!this.startRow.equals(that.startRow))
+          return false;
+      }
+
+      boolean this_present_endRow = true && this.isSetEndRow();
+      boolean that_present_endRow = true && that.isSetEndRow();
+      if (this_present_endRow || that_present_endRow) {
+        if (!(this_present_endRow && that_present_endRow))
+          return false;
+        if (!this.endRow.equals(that.endRow))
+          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 + ((isSetTableId()) ? 131071 : 524287);
+      if (isSetTableId())
+        hashCode = hashCode * 8191 + tableId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetStartRow()) ? 131071 : 524287);
+      if (isSetStartRow())
+        hashCode = hashCode * 8191 + startRow.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetEndRow()) ? 131071 : 524287);
+      if (isSetEndRow())
+        hashCode = hashCode * 8191 + endRow.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compact_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(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(isSetStartRow(), other.isSetStartRow());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetStartRow()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetEndRow(), other.isSetEndRow());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetEndRow()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.endRow, other.endRow);
+        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("compact_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("tableId:");
+      if (this.tableId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("startRow:");
+      if (this.startRow == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.startRow, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("endRow:");
+      if (this.endRow == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.endRow, 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();
+      }
+    }
+
+    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 compact_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public compact_argsStandardScheme getScheme() {
+        return new compact_argsStandardScheme();
+      }
+    }
+
+    private static class compact_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compact_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compact_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: // 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: // START_ROW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.startRow = iprot.readBinary();
+                struct.setStartRowIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // END_ROW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.endRow = iprot.readBinary();
+                struct.setEndRowIsSet(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, compact_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.tableId != null) {
+          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+          oprot.writeString(struct.tableId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.startRow != null) {
+          oprot.writeFieldBegin(START_ROW_FIELD_DESC);
+          oprot.writeBinary(struct.startRow);
+          oprot.writeFieldEnd();
+        }
+        if (struct.endRow != null) {
+          oprot.writeFieldBegin(END_ROW_FIELD_DESC);
+          oprot.writeBinary(struct.endRow);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compact_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public compact_argsTupleScheme getScheme() {
+        return new compact_argsTupleScheme();
+      }
+    }
+
+    private static class compact_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compact_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compact_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.isSetTableId()) {
+          optionals.set(3);
+        }
+        if (struct.isSetStartRow()) {
+          optionals.set(4);
+        }
+        if (struct.isSetEndRow()) {
+          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.isSetTableId()) {
+          oprot.writeString(struct.tableId);
+        }
+        if (struct.isSetStartRow()) {
+          oprot.writeBinary(struct.startRow);
+        }
+        if (struct.isSetEndRow()) {
+          oprot.writeBinary(struct.endRow);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compact_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.tableId = iprot.readString();
+          struct.setTableIdIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.startRow = iprot.readBinary();
+          struct.setStartRowIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.endRow = iprot.readBinary();
+          struct.setEndRowIsSet(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 getTabletServerStatus_args implements org.apache.thrift.TBase<getTabletServerStatus_args, getTabletServerStatus_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletServerStatus_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletServerStatus_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)3);
+    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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTabletServerStatus_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletServerStatus_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
+
+    /** 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)3, "tinfo"),
+      CREDENTIALS((short)1, "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 3: // TINFO
+            return TINFO;
+          case 1: // 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.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTabletServerStatus_args.class, metaDataMap);
+    }
+
+    public getTabletServerStatus_args() {
+    }
+
+    public getTabletServerStatus_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTabletServerStatus_args(getTabletServerStatus_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);
+      }
+    }
+
+    @Override
+    public getTabletServerStatus_args deepCopy() {
+      return new getTabletServerStatus_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getTabletServerStatus_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 getTabletServerStatus_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 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;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getTabletServerStatus_args)
+        return this.equals((getTabletServerStatus_args)that);
+      return false;
+    }
+
+    public boolean equals(getTabletServerStatus_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getTabletServerStatus_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @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("getTabletServerStatus_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getTabletServerStatus_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletServerStatus_argsStandardScheme getScheme() {
+        return new getTabletServerStatus_argsStandardScheme();
+      }
+    }
+
+    private static class getTabletServerStatus_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletServerStatus_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletServerStatus_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 3: // 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;
+            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, getTabletServerStatus_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTabletServerStatus_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletServerStatus_argsTupleScheme getScheme() {
+        return new getTabletServerStatus_argsTupleScheme();
+      }
+    }
+
+    private static class getTabletServerStatus_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletServerStatus_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletServerStatus_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTabletServerStatus_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.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 getTabletServerStatus_result implements org.apache.thrift.TBase<getTabletServerStatus_result, getTabletServerStatus_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletServerStatus_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletServerStatus_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 getTabletServerStatus_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletServerStatus_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.manager.thrift.TabletServerStatus 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.manager.thrift.TabletServerStatus.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(getTabletServerStatus_result.class, metaDataMap);
+    }
+
+    public getTabletServerStatus_result() {
+    }
+
+    public getTabletServerStatus_result(
+      org.apache.accumulo.core.manager.thrift.TabletServerStatus 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 getTabletServerStatus_result(getTabletServerStatus_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.manager.thrift.TabletServerStatus(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public getTabletServerStatus_result deepCopy() {
+      return new getTabletServerStatus_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.manager.thrift.TabletServerStatus getSuccess() {
+      return this.success;
+    }
+
+    public getTabletServerStatus_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.manager.thrift.TabletServerStatus 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 getTabletServerStatus_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.manager.thrift.TabletServerStatus)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 getTabletServerStatus_result)
+        return this.equals((getTabletServerStatus_result)that);
+      return false;
+    }
+
+    public boolean equals(getTabletServerStatus_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(getTabletServerStatus_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("getTabletServerStatus_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 getTabletServerStatus_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletServerStatus_resultStandardScheme getScheme() {
+        return new getTabletServerStatus_resultStandardScheme();
+      }
+    }
+
+    private static class getTabletServerStatus_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletServerStatus_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletServerStatus_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.manager.thrift.TabletServerStatus();
+                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, getTabletServerStatus_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 getTabletServerStatus_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletServerStatus_resultTupleScheme getScheme() {
+        return new getTabletServerStatus_resultTupleScheme();
+      }
+    }
+
+    private static class getTabletServerStatus_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletServerStatus_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletServerStatus_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, getTabletServerStatus_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.manager.thrift.TabletServerStatus();
+          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 getTabletStats_args implements org.apache.thrift.TBase<getTabletStats_args, getTabletStats_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletStats_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletStats_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)3);
+    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 TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("tableId", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTabletStats_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletStats_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 tableId; // 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)3, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      TABLE_ID((short)2, "tableId");
+
+      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 3: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // TABLE_ID
+            return TABLE_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
+    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.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTabletStats_args.class, metaDataMap);
+    }
+
+    public getTabletStats_args() {
+    }
+
+    public getTabletStats_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String tableId)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.tableId = tableId;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTabletStats_args(getTabletStats_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.isSetTableId()) {
+        this.tableId = other.tableId;
+      }
+    }
+
+    @Override
+    public getTabletStats_args deepCopy() {
+      return new getTabletStats_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.tableId = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getTabletStats_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 getTabletStats_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 getTableId() {
+      return this.tableId;
+    }
+
+    public getTabletStats_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;
+      }
+    }
+
+    @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 TABLE_ID:
+        if (value == null) {
+          unsetTableId();
+        } else {
+          setTableId((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 TABLE_ID:
+        return getTableId();
+
+      }
+      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 TABLE_ID:
+        return isSetTableId();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getTabletStats_args)
+        return this.equals((getTabletStats_args)that);
+      return false;
+    }
+
+    public boolean equals(getTabletStats_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_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;
+      }
+
+      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 + ((isSetTableId()) ? 131071 : 524287);
+      if (isSetTableId())
+        hashCode = hashCode * 8191 + tableId.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getTabletStats_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(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;
+        }
+      }
+      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("getTabletStats_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("tableId:");
+      if (this.tableId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableId);
+      }
+      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 getTabletStats_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletStats_argsStandardScheme getScheme() {
+        return new getTabletStats_argsStandardScheme();
+      }
+    }
+
+    private static class getTabletStats_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletStats_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletStats_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 3: // 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: // 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;
+            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, getTabletStats_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.tableId != null) {
+          oprot.writeFieldBegin(TABLE_ID_FIELD_DESC);
+          oprot.writeString(struct.tableId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTabletStats_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletStats_argsTupleScheme getScheme() {
+        return new getTabletStats_argsTupleScheme();
+      }
+    }
+
+    private static class getTabletStats_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletStats_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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.isSetTableId()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetTableId()) {
+          oprot.writeString(struct.tableId);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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.tableId = iprot.readString();
+          struct.setTableIdIsSet(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 getTabletStats_result implements org.apache.thrift.TBase<getTabletStats_result, getTabletStats_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTabletStats_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTabletStats_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getTabletStats_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getTabletStats_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.List<TabletStats> 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TabletStats.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(getTabletStats_result.class, metaDataMap);
+    }
+
+    public getTabletStats_result() {
+    }
+
+    public getTabletStats_result(
+      java.util.List<TabletStats> 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 getTabletStats_result(getTabletStats_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<TabletStats> __this__success = new java.util.ArrayList<TabletStats>(other.success.size());
+        for (TabletStats other_element : other.success) {
+          __this__success.add(new TabletStats(other_element));
+        }
+        this.success = __this__success;
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public getTabletStats_result deepCopy() {
+      return new getTabletStats_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<TabletStats> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(TabletStats elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<TabletStats>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<TabletStats> getSuccess() {
+      return this.success;
+    }
+
+    public getTabletStats_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TabletStats> 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 getTabletStats_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.util.List<TabletStats>)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 getTabletStats_result)
+        return this.equals((getTabletStats_result)that);
+      return false;
+    }
+
+    public boolean equals(getTabletStats_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(getTabletStats_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("getTabletStats_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
+    }
+
+    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 getTabletStats_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletStats_resultStandardScheme getScheme() {
+        return new getTabletStats_resultStandardScheme();
+      }
+    }
+
+    private static class getTabletStats_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getTabletStats_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTabletStats_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 _list72 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TabletStats>(_list72.size);
+                  @org.apache.thrift.annotation.Nullable TabletStats _elem73;
+                  for (int _i74 = 0; _i74 < _list72.size; ++_i74)
+                  {
+                    _elem73 = new TabletStats();
+                    _elem73.read(iprot);
+                    struct.success.add(_elem73);
+                  }
+                  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 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, getTabletStats_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 (TabletStats _iter75 : struct.success)
+            {
+              _iter75.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTabletStats_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getTabletStats_resultTupleScheme getScheme() {
+        return new getTabletStats_resultTupleScheme();
+      }
+    }
+
+    private static class getTabletStats_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getTabletStats_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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.writeI32(struct.success.size());
+            for (TabletStats _iter76 : struct.success)
+            {
+              _iter76.write(oprot);
+            }
+          }
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTabletStats_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 _list77 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<TabletStats>(_list77.size);
+            @org.apache.thrift.annotation.Nullable TabletStats _elem78;
+            for (int _i79 = 0; _i79 < _list77.size; ++_i79)
+            {
+              _elem78 = new TabletStats();
+              _elem78.read(iprot);
+              struct.success.add(_elem78);
+            }
+          }
+          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 getHistoricalStats_args implements org.apache.thrift.TBase<getHistoricalStats_args, getHistoricalStats_args._Fields>, java.io.Serializable, Cloneable, Comparable<getHistoricalStats_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getHistoricalStats_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getHistoricalStats_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getHistoricalStats_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
+
+    /** 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");
+
+      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;
+          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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getHistoricalStats_args.class, metaDataMap);
+    }
+
+    public getHistoricalStats_args() {
+    }
+
+    public getHistoricalStats_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getHistoricalStats_args(getHistoricalStats_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);
+      }
+    }
+
+    @Override
+    public getHistoricalStats_args deepCopy() {
+      return new getHistoricalStats_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getHistoricalStats_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 getHistoricalStats_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 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;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getHistoricalStats_args)
+        return this.equals((getHistoricalStats_args)that);
+      return false;
+    }
+
+    public boolean equals(getHistoricalStats_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getHistoricalStats_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @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("getHistoricalStats_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getHistoricalStats_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getHistoricalStats_argsStandardScheme getScheme() {
+        return new getHistoricalStats_argsStandardScheme();
+      }
+    }
+
+    private static class getHistoricalStats_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getHistoricalStats_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getHistoricalStats_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;
+            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, getHistoricalStats_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getHistoricalStats_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getHistoricalStats_argsTupleScheme getScheme() {
+        return new getHistoricalStats_argsTupleScheme();
+      }
+    }
+
+    private static class getHistoricalStats_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getHistoricalStats_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getHistoricalStats_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getHistoricalStats_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.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 getHistoricalStats_result implements org.apache.thrift.TBase<getHistoricalStats_result, getHistoricalStats_result._Fields>, java.io.Serializable, Cloneable, Comparable<getHistoricalStats_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getHistoricalStats_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 getHistoricalStats_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getHistoricalStats_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable TabletStats 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, TabletStats.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(getHistoricalStats_result.class, metaDataMap);
+    }
+
+    public getHistoricalStats_result() {
+    }
+
+    public getHistoricalStats_result(
+      TabletStats 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 getHistoricalStats_result(getHistoricalStats_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TabletStats(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public getHistoricalStats_result deepCopy() {
+      return new getHistoricalStats_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public TabletStats getSuccess() {
+      return this.success;
+    }
+
+    public getHistoricalStats_result setSuccess(@org.apache.thrift.annotation.Nullable TabletStats 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 getHistoricalStats_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((TabletStats)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 getHistoricalStats_result)
+        return this.equals((getHistoricalStats_result)that);
+      return false;
+    }
+
+    public boolean equals(getHistoricalStats_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(getHistoricalStats_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("getHistoricalStats_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 getHistoricalStats_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getHistoricalStats_resultStandardScheme getScheme() {
+        return new getHistoricalStats_resultStandardScheme();
+      }
+    }
+
+    private static class getHistoricalStats_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getHistoricalStats_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getHistoricalStats_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 TabletStats();
+                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, getHistoricalStats_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 getHistoricalStats_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getHistoricalStats_resultTupleScheme getScheme() {
+        return new getHistoricalStats_resultTupleScheme();
+      }
+    }
+
+    private static class getHistoricalStats_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getHistoricalStats_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getHistoricalStats_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, getHistoricalStats_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 TabletStats();
+          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 halt_args implements org.apache.thrift.TBase<halt_args, halt_args._Fields>, java.io.Serializable, Cloneable, Comparable<halt_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("halt_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)3);
+    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)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new halt_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new halt_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
+
+    /** 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)3, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      LOCK((short)2, "lock");
+
+      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 3: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // LOCK
+            return LOCK;
+          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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(halt_args.class, metaDataMap);
+    }
+
+    public halt_args() {
+    }
+
+    public halt_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public halt_args(halt_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;
+      }
+    }
+
+    @Override
+    public halt_args deepCopy() {
+      return new halt_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public halt_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 halt_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 halt_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;
+      }
+    }
+
+    @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;
+
+      }
+    }
+
+    @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();
+
+      }
+      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();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof halt_args)
+        return this.equals((halt_args)that);
+      return false;
+    }
+
+    public boolean equals(halt_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;
+      }
+
+      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();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(halt_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;
+        }
+      }
+      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("halt_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;
+      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 halt_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public halt_argsStandardScheme getScheme() {
+        return new halt_argsStandardScheme();
+      }
+    }
+
+    private static class halt_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<halt_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, halt_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 3: // 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: // 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;
+            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, halt_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.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 halt_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public halt_argsTupleScheme getScheme() {
+        return new halt_argsTupleScheme();
+      }
+    }
+
+    private static class halt_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<halt_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, halt_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);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetLock()) {
+          oprot.writeString(struct.lock);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, halt_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.lock = iprot.readString();
+          struct.setLockIsSet(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 halt_result implements org.apache.thrift.TBase<halt_result, halt_result._Fields>, java.io.Serializable, Cloneable, Comparable<halt_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("halt_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new halt_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new halt_resultTupleSchemeFactory();
+
+    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 {
+      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 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.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(halt_result.class, metaDataMap);
+    }
+
+    public halt_result() {
+    }
+
+    public halt_result(
+      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec)
+    {
+      this();
+      this.sec = sec;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public halt_result(halt_result other) {
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public halt_result deepCopy() {
+      return new halt_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public halt_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 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 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 SEC:
+        return isSetSec();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof halt_result)
+        return this.equals((halt_result)that);
+      return false;
+    }
+
+    public boolean equals(halt_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;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSec()) ? 131071 : 524287);
+      if (isSetSec())
+        hashCode = hashCode * 8191 + sec.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(halt_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;
+        }
+      }
+      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("halt_result(");
+      boolean first = true;
+
+      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 {
+        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 halt_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public halt_resultStandardScheme getScheme() {
+        return new halt_resultStandardScheme();
+      }
+    }
+
+    private static class halt_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<halt_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, halt_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;
+            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, halt_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class halt_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public halt_resultTupleScheme getScheme() {
+        return new halt_resultTupleScheme();
+      }
+    }
+
+    private static class halt_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<halt_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, halt_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);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, halt_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.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 fastHalt_args implements org.apache.thrift.TBase<fastHalt_args, fastHalt_args._Fields>, java.io.Serializable, Cloneable, Comparable<fastHalt_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fastHalt_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)3);
+    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)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new fastHalt_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new fastHalt_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
+
+    /** 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)3, "tinfo"),
+      CREDENTIALS((short)1, "credentials"),
+      LOCK((short)2, "lock");
+
+      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 3: // TINFO
+            return TINFO;
+          case 1: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // LOCK
+            return LOCK;
+          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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fastHalt_args.class, metaDataMap);
+    }
+
+    public fastHalt_args() {
+    }
+
+    public fastHalt_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String lock)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.lock = lock;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public fastHalt_args(fastHalt_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;
+      }
+    }
+
+    @Override
+    public fastHalt_args deepCopy() {
+      return new fastHalt_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.lock = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public fastHalt_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 fastHalt_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 fastHalt_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;
+      }
+    }
+
+    @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;
+
+      }
+    }
+
+    @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();
+
+      }
+      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();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof fastHalt_args)
+        return this.equals((fastHalt_args)that);
+      return false;
+    }
+
+    public boolean equals(fastHalt_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;
+      }
+
+      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();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(fastHalt_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;
+        }
+      }
+      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("fastHalt_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;
+      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 fastHalt_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public fastHalt_argsStandardScheme getScheme() {
+        return new fastHalt_argsStandardScheme();
+      }
+    }
+
+    private static class fastHalt_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<fastHalt_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, fastHalt_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 3: // 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: // 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;
+            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, fastHalt_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.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 fastHalt_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public fastHalt_argsTupleScheme getScheme() {
+        return new fastHalt_argsTupleScheme();
+      }
+    }
+
+    private static class fastHalt_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<fastHalt_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, fastHalt_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);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetLock()) {
+          oprot.writeString(struct.lock);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, fastHalt_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.lock = iprot.readString();
+          struct.setLockIsSet(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 getActiveCompactions_args implements org.apache.thrift.TBase<getActiveCompactions_args, getActiveCompactions_args._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveCompactions_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveCompactions_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.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.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. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)2, "tinfo"),
+      CREDENTIALS((short)1, "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 2: // TINFO
+            return TINFO;
+          case 1: // 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.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveCompactions_args.class, metaDataMap);
+    }
+
+    public getActiveCompactions_args() {
+    }
+
+    public getActiveCompactions_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getActiveCompactions_args(getActiveCompactions_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);
+      }
+    }
+
+    @Override
+    public getActiveCompactions_args deepCopy() {
+      return new getActiveCompactions_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    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.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 getActiveCompactions_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 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;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getActiveCompactions_args)
+        return this.equals((getActiveCompactions_args)that);
+      return false;
+    }
+
+    public boolean equals(getActiveCompactions_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getActiveCompactions_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @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("getActiveCompactions_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getActiveCompactions_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveCompactions_argsStandardScheme getScheme() {
+        return new getActiveCompactions_argsStandardScheme();
+      }
+    }
+
+    private static class getActiveCompactions_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveCompactions_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_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;
+            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, getActiveCompactions_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getActiveCompactions_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveCompactions_argsTupleScheme getScheme() {
+        return new getActiveCompactions_argsTupleScheme();
+      }
+    }
+
+    private static class getActiveCompactions_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveCompactions_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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.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 getActiveCompactions_result implements org.apache.thrift.TBase<getActiveCompactions_result, getActiveCompactions_result._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveCompactions_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveCompactions_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveCompactions_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveCompactions_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.List<ActiveCompaction> 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ActiveCompaction.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(getActiveCompactions_result.class, metaDataMap);
+    }
+
+    public getActiveCompactions_result() {
+    }
+
+    public getActiveCompactions_result(
+      java.util.List<ActiveCompaction> 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 getActiveCompactions_result(getActiveCompactions_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<ActiveCompaction> __this__success = new java.util.ArrayList<ActiveCompaction>(other.success.size());
+        for (ActiveCompaction other_element : other.success) {
+          __this__success.add(new ActiveCompaction(other_element));
+        }
+        this.success = __this__success;
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public getActiveCompactions_result deepCopy() {
+      return new getActiveCompactions_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<ActiveCompaction> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(ActiveCompaction elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<ActiveCompaction>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<ActiveCompaction> getSuccess() {
+      return this.success;
+    }
+
+    public getActiveCompactions_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<ActiveCompaction> 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 getActiveCompactions_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.util.List<ActiveCompaction>)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 getActiveCompactions_result)
+        return this.equals((getActiveCompactions_result)that);
+      return false;
+    }
+
+    public boolean equals(getActiveCompactions_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(getActiveCompactions_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("getActiveCompactions_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
+    }
+
+    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 getActiveCompactions_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveCompactions_resultStandardScheme getScheme() {
+        return new getActiveCompactions_resultStandardScheme();
+      }
+    }
+
+    private static class getActiveCompactions_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveCompactions_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveCompactions_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 _list80 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<ActiveCompaction>(_list80.size);
+                  @org.apache.thrift.annotation.Nullable ActiveCompaction _elem81;
+                  for (int _i82 = 0; _i82 < _list80.size; ++_i82)
+                  {
+                    _elem81 = new ActiveCompaction();
+                    _elem81.read(iprot);
+                    struct.success.add(_elem81);
+                  }
+                  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 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, getActiveCompactions_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 (ActiveCompaction _iter83 : struct.success)
+            {
+              _iter83.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getActiveCompactions_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveCompactions_resultTupleScheme getScheme() {
+        return new getActiveCompactions_resultTupleScheme();
+      }
+    }
+
+    private static class getActiveCompactions_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveCompactions_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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.writeI32(struct.success.size());
+            for (ActiveCompaction _iter84 : struct.success)
+            {
+              _iter84.write(oprot);
+            }
+          }
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveCompactions_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 _list85 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<ActiveCompaction>(_list85.size);
+            @org.apache.thrift.annotation.Nullable ActiveCompaction _elem86;
+            for (int _i87 = 0; _i87 < _list85.size; ++_i87)
+            {
+              _elem86 = new ActiveCompaction();
+              _elem86.read(iprot);
+              struct.success.add(_elem86);
+            }
+          }
+          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 removeLogs_args implements org.apache.thrift.TBase<removeLogs_args, removeLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<removeLogs_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("removeLogs_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 FILENAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("filenames", org.apache.thrift.protocol.TType.LIST, (short)3);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new removeLogs_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new removeLogs_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.lang.String> filenames; // 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"),
+      FILENAMES((short)3, "filenames");
+
+      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: // FILENAMES
+            return FILENAMES;
+          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.FILENAMES, new org.apache.thrift.meta_data.FieldMetaData("filenames", 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(removeLogs_args.class, metaDataMap);
+    }
+
+    public removeLogs_args() {
+    }
+
+    public removeLogs_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.util.List<java.lang.String> filenames)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.filenames = filenames;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public removeLogs_args(removeLogs_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.isSetFilenames()) {
+        java.util.List<java.lang.String> __this__filenames = new java.util.ArrayList<java.lang.String>(other.filenames);
+        this.filenames = __this__filenames;
+      }
+    }
+
+    @Override
+    public removeLogs_args deepCopy() {
+      return new removeLogs_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.filenames = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public removeLogs_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 removeLogs_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 getFilenamesSize() {
+      return (this.filenames == null) ? 0 : this.filenames.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<java.lang.String> getFilenamesIterator() {
+      return (this.filenames == null) ? null : this.filenames.iterator();
+    }
+
+    public void addToFilenames(java.lang.String elem) {
+      if (this.filenames == null) {
+        this.filenames = new java.util.ArrayList<java.lang.String>();
+      }
+      this.filenames.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<java.lang.String> getFilenames() {
+      return this.filenames;
+    }
+
+    public removeLogs_args setFilenames(@org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> filenames) {
+      this.filenames = filenames;
+      return this;
+    }
+
+    public void unsetFilenames() {
+      this.filenames = null;
+    }
+
+    /** Returns true if field filenames is set (has been assigned a value) and false otherwise */
+    public boolean isSetFilenames() {
+      return this.filenames != null;
+    }
+
+    public void setFilenamesIsSet(boolean value) {
+      if (!value) {
+        this.filenames = 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 FILENAMES:
+        if (value == null) {
+          unsetFilenames();
+        } else {
+          setFilenames((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 CREDENTIALS:
+        return getCredentials();
+
+      case FILENAMES:
+        return getFilenames();
+
+      }
+      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 FILENAMES:
+        return isSetFilenames();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof removeLogs_args)
+        return this.equals((removeLogs_args)that);
+      return false;
+    }
+
+    public boolean equals(removeLogs_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_filenames = true && this.isSetFilenames();
+      boolean that_present_filenames = true && that.isSetFilenames();
+      if (this_present_filenames || that_present_filenames) {
+        if (!(this_present_filenames && that_present_filenames))
+          return false;
+        if (!this.filenames.equals(that.filenames))
+          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 + ((isSetFilenames()) ? 131071 : 524287);
+      if (isSetFilenames())
+        hashCode = hashCode * 8191 + filenames.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(removeLogs_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(isSetFilenames(), other.isSetFilenames());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetFilenames()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filenames, other.filenames);
+        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("removeLogs_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("filenames:");
+      if (this.filenames == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.filenames);
+      }
+      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 removeLogs_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public removeLogs_argsStandardScheme getScheme() {
+        return new removeLogs_argsStandardScheme();
+      }
+    }
+
+    private static class removeLogs_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<removeLogs_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, removeLogs_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: // FILENAMES
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list88 = iprot.readListBegin();
+                  struct.filenames = new java.util.ArrayList<java.lang.String>(_list88.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem89;
+                  for (int _i90 = 0; _i90 < _list88.size; ++_i90)
+                  {
+                    _elem89 = iprot.readString();
+                    struct.filenames.add(_elem89);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setFilenamesIsSet(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, removeLogs_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.filenames != null) {
+          oprot.writeFieldBegin(FILENAMES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filenames.size()));
+            for (java.lang.String _iter91 : struct.filenames)
+            {
+              oprot.writeString(_iter91);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class removeLogs_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public removeLogs_argsTupleScheme getScheme() {
+        return new removeLogs_argsTupleScheme();
+      }
+    }
+
+    private static class removeLogs_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<removeLogs_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, removeLogs_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.isSetFilenames()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetFilenames()) {
+          {
+            oprot.writeI32(struct.filenames.size());
+            for (java.lang.String _iter92 : struct.filenames)
+            {
+              oprot.writeString(_iter92);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_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)) {
+          {
+            org.apache.thrift.protocol.TList _list93 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.filenames = new java.util.ArrayList<java.lang.String>(_list93.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem94;
+            for (int _i95 = 0; _i95 < _list93.size; ++_i95)
+            {
+              _elem94 = iprot.readString();
+              struct.filenames.add(_elem94);
+            }
+          }
+          struct.setFilenamesIsSet(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 getActiveLogs_args implements org.apache.thrift.TBase<getActiveLogs_args, getActiveLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveLogs_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveLogs_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
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      @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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveLogs_args.class, metaDataMap);
+    }
+
+    public getActiveLogs_args() {
+    }
+
+    public getActiveLogs_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getActiveLogs_args(getActiveLogs_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);
+      }
+    }
+
+    @Override
+    public getActiveLogs_args deepCopy() {
+      return new getActiveLogs_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getActiveLogs_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 getActiveLogs_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 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;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getActiveLogs_args)
+        return this.equals((getActiveLogs_args)that);
+      return false;
+    }
+
+    public boolean equals(getActiveLogs_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getActiveLogs_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @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("getActiveLogs_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getActiveLogs_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveLogs_argsStandardScheme getScheme() {
+        return new getActiveLogs_argsStandardScheme();
+      }
+    }
+
+    private static class getActiveLogs_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveLogs_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_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;
+            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, getActiveLogs_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getActiveLogs_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveLogs_argsTupleScheme getScheme() {
+        return new getActiveLogs_argsTupleScheme();
+      }
+    }
+
+    private static class getActiveLogs_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveLogs_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_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.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 getActiveLogs_result implements org.apache.thrift.TBase<getActiveLogs_result, getActiveLogs_result._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getActiveLogs_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getActiveLogs_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.List<java.lang.String> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      @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))));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getActiveLogs_result.class, metaDataMap);
+    }
+
+    public getActiveLogs_result() {
+    }
+
+    public getActiveLogs_result(
+      java.util.List<java.lang.String> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getActiveLogs_result(getActiveLogs_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;
+      }
+    }
+
+    @Override
+    public getActiveLogs_result deepCopy() {
+      return new getActiveLogs_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = 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 getActiveLogs_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;
+      }
+    }
+
+    @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;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @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 getActiveLogs_result)
+        return this.equals((getActiveLogs_result)that);
+      return false;
+    }
+
+    public boolean equals(getActiveLogs_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getActiveLogs_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("getActiveLogs_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getActiveLogs_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveLogs_resultStandardScheme getScheme() {
+        return new getActiveLogs_resultStandardScheme();
+      }
+    }
+
+    private static class getActiveLogs_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getActiveLogs_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getActiveLogs_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 _list96 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<java.lang.String>(_list96.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _elem97;
+                  for (int _i98 = 0; _i98 < _list96.size; ++_i98)
+                  {
+                    _elem97 = iprot.readString();
+                    struct.success.add(_elem97);
+                  }
+                  iprot.readListEnd();
+                }
+                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, getActiveLogs_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 _iter99 : struct.success)
+            {
+              oprot.writeString(_iter99);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getActiveLogs_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getActiveLogs_resultTupleScheme getScheme() {
+        return new getActiveLogs_resultTupleScheme();
+      }
+    }
+
+    private static class getActiveLogs_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getActiveLogs_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (java.lang.String _iter100 : struct.success)
+            {
+              oprot.writeString(_iter100);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getActiveLogs_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list101 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRING);
+            struct.success = new java.util.ArrayList<java.lang.String>(_list101.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _elem102;
+            for (int _i103 = 0; _i103 < _list101.size; ++_i103)
+            {
+              _elem102 = iprot.readString();
+              struct.success.add(_elem102);
+            }
+          }
+          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 startGetSummaries_args implements org.apache.thrift.TBase<startGetSummaries_args, startGetSummaries_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummaries_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummaries_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 REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummaries_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummaries_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.TSummaryRequest request; // 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"),
+      REQUEST((short)3, "request");
+
+      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: // REQUEST
+            return REQUEST;
+          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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummaries_args.class, metaDataMap);
+    }
+
+    public startGetSummaries_args() {
+    }
+
+    public startGetSummaries_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.request = request;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startGetSummaries_args(startGetSummaries_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.isSetRequest()) {
+        this.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest(other.request);
+      }
+    }
+
+    @Override
+    public startGetSummaries_args deepCopy() {
+      return new startGetSummaries_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.request = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startGetSummaries_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 startGetSummaries_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.TSummaryRequest getRequest() {
+      return this.request;
+    }
+
+    public startGetSummaries_args setRequest(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) {
+      this.request = request;
+      return this;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = 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 REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest)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 REQUEST:
+        return getRequest();
+
+      }
+      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 REQUEST:
+        return isSetRequest();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startGetSummaries_args)
+        return this.equals((startGetSummaries_args)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummaries_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_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          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 + ((isSetRequest()) ? 131071 : 524287);
+      if (isSetRequest())
+        hashCode = hashCode * 8191 + request.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startGetSummaries_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(isSetRequest(), other.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
+        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("startGetSummaries_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("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      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 (request != null) {
+        request.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 startGetSummaries_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummaries_argsStandardScheme getScheme() {
+        return new startGetSummaries_argsStandardScheme();
+      }
+    }
+
+    private static class startGetSummaries_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummaries_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummaries_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: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(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, startGetSummaries_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.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startGetSummaries_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummaries_argsTupleScheme getScheme() {
+        return new startGetSummaries_argsTupleScheme();
+      }
+    }
+
+    private static class startGetSummaries_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummaries_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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.isSetRequest()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(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 startGetSummaries_result implements org.apache.thrift.TBase<startGetSummaries_result, startGetSummaries_result._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummaries_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummaries_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.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 startGetSummaries_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummaries_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries success; // required
+    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.clientImpl.thrift.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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaries.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.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, org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummaries_result.class, metaDataMap);
+    }
+
+    public startGetSummaries_result() {
+    }
+
+    public startGetSummaries_result(
+      org.apache.accumulo.core.dataImpl.thrift.TSummaries success,
+      org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException sec,
+      org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException tope)
+    {
+      this();
+      this.success = success;
+      this.sec = sec;
+      this.tope = tope;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startGetSummaries_result(startGetSummaries_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+      if (other.isSetTope()) {
+        this.tope = new org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException(other.tope);
+      }
+    }
+
+    @Override
+    public startGetSummaries_result deepCopy() {
+      return new startGetSummaries_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+      this.tope = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
+      return this.success;
+    }
+
+    public startGetSummaries_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummaries_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.clientImpl.thrift.ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public startGetSummaries_result setTope(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.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((org.apache.accumulo.core.dataImpl.thrift.TSummaries)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException)value);
+        }
+        break;
+
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((org.apache.accumulo.core.clientImpl.thrift.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 startGetSummaries_result)
+        return this.equals((startGetSummaries_result)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummaries_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(startGetSummaries_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("startGetSummaries_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
+      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 startGetSummaries_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummaries_resultStandardScheme getScheme() {
+        return new startGetSummaries_resultStandardScheme();
+      }
+    }
+
+    private static class startGetSummaries_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummaries_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummaries_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.TSummaries();
+                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;
+            case 2: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new org.apache.accumulo.core.clientImpl.thrift.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, startGetSummaries_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();
+        }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startGetSummaries_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummaries_resultTupleScheme getScheme() {
+        return new startGetSummaries_resultTupleScheme();
+      }
+    }
+
+    private static class startGetSummaries_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummaries_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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()) {
+          struct.success.write(oprot);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummaries_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)) {
+          struct.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries();
+          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);
+        }
+        if (incoming.get(2)) {
+          struct.tope = new org.apache.accumulo.core.clientImpl.thrift.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 startGetSummariesForPartition_args implements org.apache.thrift.TBase<startGetSummariesForPartition_args, startGetSummariesForPartition_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesForPartition_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesForPartition_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 REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField MODULUS_FIELD_DESC = new org.apache.thrift.protocol.TField("modulus", org.apache.thrift.protocol.TType.I32, (short)4);
+    private static final org.apache.thrift.protocol.TField REMAINDER_FIELD_DESC = new org.apache.thrift.protocol.TField("remainder", org.apache.thrift.protocol.TType.I32, (short)5);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummariesForPartition_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesForPartition_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.TSummaryRequest request; // required
+    public int modulus; // required
+    public int remainder; // 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"),
+      REQUEST((short)3, "request"),
+      MODULUS((short)4, "modulus"),
+      REMAINDER((short)5, "remainder");
+
+      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: // REQUEST
+            return REQUEST;
+          case 4: // MODULUS
+            return MODULUS;
+          case 5: // REMAINDER
+            return REMAINDER;
+          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 __MODULUS_ISSET_ID = 0;
+    private static final int __REMAINDER_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.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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest.class)));
+      tmpMap.put(_Fields.MODULUS, new org.apache.thrift.meta_data.FieldMetaData("modulus", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+      tmpMap.put(_Fields.REMAINDER, new org.apache.thrift.meta_data.FieldMetaData("remainder", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummariesForPartition_args.class, metaDataMap);
+    }
+
+    public startGetSummariesForPartition_args() {
+    }
+
+    public startGetSummariesForPartition_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request,
+      int modulus,
+      int remainder)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.request = request;
+      this.modulus = modulus;
+      setModulusIsSet(true);
+      this.remainder = remainder;
+      setRemainderIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startGetSummariesForPartition_args(startGetSummariesForPartition_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.isSetRequest()) {
+        this.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest(other.request);
+      }
+      this.modulus = other.modulus;
+      this.remainder = other.remainder;
+    }
+
+    @Override
+    public startGetSummariesForPartition_args deepCopy() {
+      return new startGetSummariesForPartition_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.request = null;
+      setModulusIsSet(false);
+      this.modulus = 0;
+      setRemainderIsSet(false);
+      this.remainder = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startGetSummariesForPartition_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 startGetSummariesForPartition_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.TSummaryRequest getRequest() {
+      return this.request;
+    }
+
+    public startGetSummariesForPartition_args setRequest(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) {
+      this.request = request;
+      return this;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public int getModulus() {
+      return this.modulus;
+    }
+
+    public startGetSummariesForPartition_args setModulus(int modulus) {
+      this.modulus = modulus;
+      setModulusIsSet(true);
+      return this;
+    }
+
+    public void unsetModulus() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __MODULUS_ISSET_ID);
+    }
+
+    /** Returns true if field modulus is set (has been assigned a value) and false otherwise */
+    public boolean isSetModulus() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __MODULUS_ISSET_ID);
+    }
+
+    public void setModulusIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __MODULUS_ISSET_ID, value);
+    }
+
+    public int getRemainder() {
+      return this.remainder;
+    }
+
+    public startGetSummariesForPartition_args setRemainder(int remainder) {
+      this.remainder = remainder;
+      setRemainderIsSet(true);
+      return this;
+    }
+
+    public void unsetRemainder() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __REMAINDER_ISSET_ID);
+    }
+
+    /** Returns true if field remainder is set (has been assigned a value) and false otherwise */
+    public boolean isSetRemainder() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __REMAINDER_ISSET_ID);
+    }
+
+    public void setRemainderIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __REMAINDER_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 REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest)value);
+        }
+        break;
+
+      case MODULUS:
+        if (value == null) {
+          unsetModulus();
+        } else {
+          setModulus((java.lang.Integer)value);
+        }
+        break;
+
+      case REMAINDER:
+        if (value == null) {
+          unsetRemainder();
+        } else {
+          setRemainder((java.lang.Integer)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 REQUEST:
+        return getRequest();
+
+      case MODULUS:
+        return getModulus();
+
+      case REMAINDER:
+        return getRemainder();
+
+      }
+      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 REQUEST:
+        return isSetRequest();
+      case MODULUS:
+        return isSetModulus();
+      case REMAINDER:
+        return isSetRemainder();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startGetSummariesForPartition_args)
+        return this.equals((startGetSummariesForPartition_args)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummariesForPartition_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_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          return false;
+      }
+
+      boolean this_present_modulus = true;
+      boolean that_present_modulus = true;
+      if (this_present_modulus || that_present_modulus) {
+        if (!(this_present_modulus && that_present_modulus))
+          return false;
+        if (this.modulus != that.modulus)
+          return false;
+      }
+
+      boolean this_present_remainder = true;
+      boolean that_present_remainder = true;
+      if (this_present_remainder || that_present_remainder) {
+        if (!(this_present_remainder && that_present_remainder))
+          return false;
+        if (this.remainder != that.remainder)
+          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 + ((isSetRequest()) ? 131071 : 524287);
+      if (isSetRequest())
+        hashCode = hashCode * 8191 + request.hashCode();
+
+      hashCode = hashCode * 8191 + modulus;
+
+      hashCode = hashCode * 8191 + remainder;
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startGetSummariesForPartition_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(isSetRequest(), other.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetModulus(), other.isSetModulus());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetModulus()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.modulus, other.modulus);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetRemainder(), other.isSetRemainder());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRemainder()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remainder, other.remainder);
+        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("startGetSummariesForPartition_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("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("modulus:");
+      sb.append(this.modulus);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("remainder:");
+      sb.append(this.remainder);
+      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 (request != null) {
+        request.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 startGetSummariesForPartition_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesForPartition_argsStandardScheme getScheme() {
+        return new startGetSummariesForPartition_argsStandardScheme();
+      }
+    }
+
+    private static class startGetSummariesForPartition_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesForPartition_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesForPartition_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: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // MODULUS
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.modulus = iprot.readI32();
+                struct.setModulusIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // REMAINDER
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.remainder = iprot.readI32();
+                struct.setRemainderIsSet(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, startGetSummariesForPartition_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.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(MODULUS_FIELD_DESC);
+        oprot.writeI32(struct.modulus);
+        oprot.writeFieldEnd();
+        oprot.writeFieldBegin(REMAINDER_FIELD_DESC);
+        oprot.writeI32(struct.remainder);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startGetSummariesForPartition_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesForPartition_argsTupleScheme getScheme() {
+        return new startGetSummariesForPartition_argsTupleScheme();
+      }
+    }
+
+    private static class startGetSummariesForPartition_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesForPartition_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesForPartition_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.isSetRequest()) {
+          optionals.set(2);
+        }
+        if (struct.isSetModulus()) {
+          optionals.set(3);
+        }
+        if (struct.isSetRemainder()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+        if (struct.isSetModulus()) {
+          oprot.writeI32(struct.modulus);
+        }
+        if (struct.isSetRemainder()) {
+          oprot.writeI32(struct.remainder);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesForPartition_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(5);
+        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.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.modulus = iprot.readI32();
+          struct.setModulusIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.remainder = iprot.readI32();
+          struct.setRemainderIsSet(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 startGetSummariesForPartition_result implements org.apache.thrift.TBase<startGetSummariesForPartition_result, startGetSummariesForPartition_result._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesForPartition_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesForPartition_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 startGetSummariesForPartition_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesForPartition_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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.TSummaries.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(startGetSummariesForPartition_result.class, metaDataMap);
+    }
+
+    public startGetSummariesForPartition_result() {
+    }
+
+    public startGetSummariesForPartition_result(
+      org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesForPartition_result(startGetSummariesForPartition_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public startGetSummariesForPartition_result deepCopy() {
+      return new startGetSummariesForPartition_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
+      return this.success;
+    }
+
+    public startGetSummariesForPartition_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesForPartition_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.TSummaries)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 startGetSummariesForPartition_result)
+        return this.equals((startGetSummariesForPartition_result)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummariesForPartition_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(startGetSummariesForPartition_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("startGetSummariesForPartition_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 startGetSummariesForPartition_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesForPartition_resultStandardScheme getScheme() {
+        return new startGetSummariesForPartition_resultStandardScheme();
+      }
+    }
+
+    private static class startGetSummariesForPartition_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesForPartition_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesForPartition_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.TSummaries();
+                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, startGetSummariesForPartition_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 startGetSummariesForPartition_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesForPartition_resultTupleScheme getScheme() {
+        return new startGetSummariesForPartition_resultTupleScheme();
+      }
+    }
+
+    private static class startGetSummariesForPartition_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesForPartition_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesForPartition_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, startGetSummariesForPartition_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.TSummaries();
+          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 startGetSummariesFromFiles_args implements org.apache.thrift.TBase<startGetSummariesFromFiles_args, startGetSummariesFromFiles_args._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesFromFiles_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesFromFiles_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 REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+    private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.MAP, (short)4);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new startGetSummariesFromFiles_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesFromFiles_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.TSummaryRequest request; // required
+    public @org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files; // 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"),
+      REQUEST((short)3, "request"),
+      FILES((short)4, "files");
+
+      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: // REQUEST
+            return REQUEST;
+          case 4: // FILES
+            return FILES;
+          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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest.class)));
+      tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", 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.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.TRowRange.class)))));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(startGetSummariesFromFiles_args.class, metaDataMap);
+    }
+
+    public startGetSummariesFromFiles_args() {
+    }
+
+    public startGetSummariesFromFiles_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request,
+      java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> files)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.request = request;
+      this.files = files;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public startGetSummariesFromFiles_args(startGetSummariesFromFiles_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.isSetRequest()) {
+        this.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest(other.request);
+      }
+      if (other.isSetFiles()) {
+        java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> __this__files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>(other.files.size());
+        for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> other_element : other.files.entrySet()) {
+
+          java.lang.String other_element_key = other_element.getKey();
+          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> other_element_value = other_element.getValue();
+
+          java.lang.String __this__files_copy_key = other_element_key;
+
+          java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> __this__files_copy_value = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRowRange>(other_element_value.size());
+          for (org.apache.accumulo.core.dataImpl.thrift.TRowRange other_element_value_element : other_element_value) {
+            __this__files_copy_value.add(new org.apache.accumulo.core.dataImpl.thrift.TRowRange(other_element_value_element));
+          }
+
+          __this__files.put(__this__files_copy_key, __this__files_copy_value);
+        }
+        this.files = __this__files;
+      }
+    }
+
+    @Override
+    public startGetSummariesFromFiles_args deepCopy() {
+      return new startGetSummariesFromFiles_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.request = null;
+      this.files = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public startGetSummariesFromFiles_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 startGetSummariesFromFiles_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.TSummaryRequest getRequest() {
+      return this.request;
+    }
+
+    public startGetSummariesFromFiles_args setRequest(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest request) {
+      this.request = request;
+      return this;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public int getFilesSize() {
+      return (this.files == null) ? 0 : this.files.size();
+    }
+
+    public void putToFiles(java.lang.String key, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> val) {
+      if (this.files == null) {
+        this.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>();
+      }
+      this.files.put(key, val);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> getFiles() {
+      return this.files;
+    }
+
+    public startGetSummariesFromFiles_args setFiles(@org.apache.thrift.annotation.Nullable java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> 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;
+      }
+    }
+
+    @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 REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest)value);
+        }
+        break;
+
+      case FILES:
+        if (value == null) {
+          unsetFiles();
+        } else {
+          setFiles((java.util.Map<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>)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 REQUEST:
+        return getRequest();
+
+      case FILES:
+        return getFiles();
+
+      }
+      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 REQUEST:
+        return isSetRequest();
+      case FILES:
+        return isSetFiles();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof startGetSummariesFromFiles_args)
+        return this.equals((startGetSummariesFromFiles_args)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummariesFromFiles_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_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          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;
+      }
+
+      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 + ((isSetRequest()) ? 131071 : 524287);
+      if (isSetRequest())
+        hashCode = hashCode * 8191 + request.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetFiles()) ? 131071 : 524287);
+      if (isSetFiles())
+        hashCode = hashCode * 8191 + files.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(startGetSummariesFromFiles_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(isSetRequest(), other.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, other.request);
+        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;
+        }
+      }
+      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("startGetSummariesFromFiles_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("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("files:");
+      if (this.files == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.files);
+      }
+      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 (request != null) {
+        request.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 startGetSummariesFromFiles_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesFromFiles_argsStandardScheme getScheme() {
+        return new startGetSummariesFromFiles_argsStandardScheme();
+      }
+    }
+
+    private static class startGetSummariesFromFiles_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesFromFiles_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesFromFiles_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: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // FILES
+              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+                {
+                  org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin();
+                  struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>(2*_map104.size);
+                  @org.apache.thrift.annotation.Nullable java.lang.String _key105;
+                  @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> _val106;
+                  for (int _i107 = 0; _i107 < _map104.size; ++_i107)
+                  {
+                    _key105 = iprot.readString();
+                    {
+                      org.apache.thrift.protocol.TList _list108 = iprot.readListBegin();
+                      _val106 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRowRange>(_list108.size);
+                      @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem109;
+                      for (int _i110 = 0; _i110 < _list108.size; ++_i110)
+                      {
+                        _elem109 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange();
+                        _elem109.read(iprot);
+                        _val106.add(_elem109);
+                      }
+                      iprot.readListEnd();
+                    }
+                    struct.files.put(_key105, _val106);
+                  }
+                  iprot.readMapEnd();
+                }
+                struct.setFilesIsSet(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, startGetSummariesFromFiles_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.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.files != null) {
+          oprot.writeFieldBegin(FILES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.files.size()));
+            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> _iter111 : struct.files.entrySet())
+            {
+              oprot.writeString(_iter111.getKey());
+              {
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter111.getValue().size()));
+                for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter112 : _iter111.getValue())
+                {
+                  _iter112.write(oprot);
+                }
+                oprot.writeListEnd();
+              }
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class startGetSummariesFromFiles_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesFromFiles_argsTupleScheme getScheme() {
+        return new startGetSummariesFromFiles_argsTupleScheme();
+      }
+    }
+
+    private static class startGetSummariesFromFiles_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesFromFiles_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFromFiles_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.isSetRequest()) {
+          optionals.set(2);
+        }
+        if (struct.isSetFiles()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+        if (struct.isSetFiles()) {
+          {
+            oprot.writeI32(struct.files.size());
+            for (java.util.Map.Entry<java.lang.String, java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>> _iter113 : struct.files.entrySet())
+            {
+              oprot.writeString(_iter113.getKey());
+              {
+                oprot.writeI32(_iter113.getValue().size());
+                for (org.apache.accumulo.core.dataImpl.thrift.TRowRange _iter114 : _iter113.getValue())
+                {
+                  _iter114.write(oprot);
+                }
+              }
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFromFiles_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.request = new org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(true);
+        }
+        if (incoming.get(3)) {
+          {
+            org.apache.thrift.protocol.TMap _map115 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST); 
+            struct.files = new java.util.HashMap<java.lang.String,java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange>>(2*_map115.size);
+            @org.apache.thrift.annotation.Nullable java.lang.String _key116;
+            @org.apache.thrift.annotation.Nullable java.util.List<org.apache.accumulo.core.dataImpl.thrift.TRowRange> _val117;
+            for (int _i118 = 0; _i118 < _map115.size; ++_i118)
+            {
+              _key116 = iprot.readString();
+              {
+                org.apache.thrift.protocol.TList _list119 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+                _val117 = new java.util.ArrayList<org.apache.accumulo.core.dataImpl.thrift.TRowRange>(_list119.size);
+                @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TRowRange _elem120;
+                for (int _i121 = 0; _i121 < _list119.size; ++_i121)
+                {
+                  _elem120 = new org.apache.accumulo.core.dataImpl.thrift.TRowRange();
+                  _elem120.read(iprot);
+                  _val117.add(_elem120);
+                }
+              }
+              struct.files.put(_key116, _val117);
+            }
+          }
+          struct.setFilesIsSet(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 startGetSummariesFromFiles_result implements org.apache.thrift.TBase<startGetSummariesFromFiles_result, startGetSummariesFromFiles_result._Fields>, java.io.Serializable, Cloneable, Comparable<startGetSummariesFromFiles_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("startGetSummariesFromFiles_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 startGetSummariesFromFiles_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new startGetSummariesFromFiles_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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.TSummaries.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(startGetSummariesFromFiles_result.class, metaDataMap);
+    }
+
+    public startGetSummariesFromFiles_result() {
+    }
+
+    public startGetSummariesFromFiles_result(
+      org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesFromFiles_result(startGetSummariesFromFiles_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public startGetSummariesFromFiles_result deepCopy() {
+      return new startGetSummariesFromFiles_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
+      return this.success;
+    }
+
+    public startGetSummariesFromFiles_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 startGetSummariesFromFiles_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.TSummaries)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 startGetSummariesFromFiles_result)
+        return this.equals((startGetSummariesFromFiles_result)that);
+      return false;
+    }
+
+    public boolean equals(startGetSummariesFromFiles_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(startGetSummariesFromFiles_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("startGetSummariesFromFiles_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 startGetSummariesFromFiles_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesFromFiles_resultStandardScheme getScheme() {
+        return new startGetSummariesFromFiles_resultStandardScheme();
+      }
+    }
+
+    private static class startGetSummariesFromFiles_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<startGetSummariesFromFiles_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, startGetSummariesFromFiles_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.TSummaries();
+                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, startGetSummariesFromFiles_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 startGetSummariesFromFiles_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public startGetSummariesFromFiles_resultTupleScheme getScheme() {
+        return new startGetSummariesFromFiles_resultTupleScheme();
+      }
+    }
+
+    private static class startGetSummariesFromFiles_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<startGetSummariesFromFiles_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, startGetSummariesFromFiles_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, startGetSummariesFromFiles_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.TSummaries();
+          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 contiuneGetSummaries_args implements org.apache.thrift.TBase<contiuneGetSummaries_args, contiuneGetSummaries_args._Fields>, java.io.Serializable, Cloneable, Comparable<contiuneGetSummaries_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("contiuneGetSummaries_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 SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.I64, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new contiuneGetSummaries_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new contiuneGetSummaries_argsTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo; // required
+    public long sessionId; // 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"),
+      SESSION_ID((short)2, "sessionId");
+
+      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: // SESSION_ID
+            return SESSION_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 __SESSIONID_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.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", 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(contiuneGetSummaries_args.class, metaDataMap);
+    }
+
+    public contiuneGetSummaries_args() {
+    }
+
+    public contiuneGetSummaries_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      long sessionId)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.sessionId = sessionId;
+      setSessionIdIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public contiuneGetSummaries_args(contiuneGetSummaries_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.clientImpl.thrift.TInfo(other.tinfo);
+      }
+      this.sessionId = other.sessionId;
+    }
+
+    @Override
+    public contiuneGetSummaries_args deepCopy() {
+      return new contiuneGetSummaries_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      setSessionIdIsSet(false);
+      this.sessionId = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public contiuneGetSummaries_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 getSessionId() {
+      return this.sessionId;
+    }
+
+    public contiuneGetSummaries_args setSessionId(long sessionId) {
+      this.sessionId = sessionId;
+      setSessionIdIsSet(true);
+      return this;
+    }
+
+    public void unsetSessionId() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+    }
+
+    /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetSessionId() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __SESSIONID_ISSET_ID);
+    }
+
+    public void setSessionIdIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __SESSIONID_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 SESSION_ID:
+        if (value == null) {
+          unsetSessionId();
+        } else {
+          setSessionId((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 SESSION_ID:
+        return getSessionId();
+
+      }
+      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 SESSION_ID:
+        return isSetSessionId();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof contiuneGetSummaries_args)
+        return this.equals((contiuneGetSummaries_args)that);
+      return false;
+    }
+
+    public boolean equals(contiuneGetSummaries_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_sessionId = true;
+      boolean that_present_sessionId = true;
+      if (this_present_sessionId || that_present_sessionId) {
+        if (!(this_present_sessionId && that_present_sessionId))
+          return false;
+        if (this.sessionId != that.sessionId)
+          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(sessionId);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(contiuneGetSummaries_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(isSetSessionId(), other.isSetSessionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSessionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, other.sessionId);
+        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("contiuneGetSummaries_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("sessionId:");
+      sb.append(this.sessionId);
+      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 contiuneGetSummaries_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public contiuneGetSummaries_argsStandardScheme getScheme() {
+        return new contiuneGetSummaries_argsStandardScheme();
+      }
+    }
+
+    private static class contiuneGetSummaries_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<contiuneGetSummaries_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, contiuneGetSummaries_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: // SESSION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.sessionId = iprot.readI64();
+                struct.setSessionIdIsSet(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, contiuneGetSummaries_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(SESSION_ID_FIELD_DESC);
+        oprot.writeI64(struct.sessionId);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class contiuneGetSummaries_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public contiuneGetSummaries_argsTupleScheme getScheme() {
+        return new contiuneGetSummaries_argsTupleScheme();
+      }
+    }
+
+    private static class contiuneGetSummaries_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<contiuneGetSummaries_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, contiuneGetSummaries_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.isSetSessionId()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetSessionId()) {
+          oprot.writeI64(struct.sessionId);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, contiuneGetSummaries_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.sessionId = iprot.readI64();
+          struct.setSessionIdIsSet(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 contiuneGetSummaries_result implements org.apache.thrift.TBase<contiuneGetSummaries_result, contiuneGetSummaries_result._Fields>, java.io.Serializable, Cloneable, Comparable<contiuneGetSummaries_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("contiuneGetSummaries_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 contiuneGetSummaries_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new contiuneGetSummaries_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries success; // required
+    public @org.apache.thrift.annotation.Nullable 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.TSummaries.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, NoSuchScanIDException.class)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(contiuneGetSummaries_result.class, metaDataMap);
+    }
+
+    public contiuneGetSummaries_result() {
+    }
+
+    public contiuneGetSummaries_result(
+      org.apache.accumulo.core.dataImpl.thrift.TSummaries success,
+      NoSuchScanIDException nssi)
+    {
+      this();
+      this.success = success;
+      this.nssi = nssi;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public contiuneGetSummaries_result(contiuneGetSummaries_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new org.apache.accumulo.core.dataImpl.thrift.TSummaries(other.success);
+      }
+      if (other.isSetNssi()) {
+        this.nssi = new NoSuchScanIDException(other.nssi);
+      }
+    }
+
+    @Override
+    public contiuneGetSummaries_result deepCopy() {
+      return new contiuneGetSummaries_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.nssi = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TSummaries getSuccess() {
+      return this.success;
+    }
+
+    public contiuneGetSummaries_result setSuccess(@org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TSummaries 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 NoSuchScanIDException getNssi() {
+      return this.nssi;
+    }
+
+    public contiuneGetSummaries_result setNssi(@org.apache.thrift.annotation.Nullable 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.TSummaries)value);
+        }
+        break;
+
+      case NSSI:
+        if (value == null) {
+          unsetNssi();
+        } else {
+          setNssi((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 contiuneGetSummaries_result)
+        return this.equals((contiuneGetSummaries_result)that);
+      return false;
+    }
+
+    public boolean equals(contiuneGetSummaries_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(contiuneGetSummaries_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("contiuneGetSummaries_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 contiuneGetSummaries_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public contiuneGetSummaries_resultStandardScheme getScheme() {
+        return new contiuneGetSummaries_resultStandardScheme();
+      }
+    }
+
+    private static class contiuneGetSummaries_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<contiuneGetSummaries_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, contiuneGetSummaries_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.TSummaries();
+                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 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, contiuneGetSummaries_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 contiuneGetSummaries_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public contiuneGetSummaries_resultTupleScheme getScheme() {
+        return new contiuneGetSummaries_resultTupleScheme();
+      }
+    }
+
+    private static class contiuneGetSummaries_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<contiuneGetSummaries_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, contiuneGetSummaries_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, contiuneGetSummaries_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.TSummaries();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.nssi = new 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 getCompactionQueueInfo_args implements org.apache.thrift.TBase<getCompactionQueueInfo_args, getCompactionQueueInfo_args._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionQueueInfo_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionQueueInfo_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionQueueInfo_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionQueueInfo_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
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new java.lang.IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      @org.apache.thrift.annotation.Nullable
+      public static _Fields findByName(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      @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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getCompactionQueueInfo_args.class, metaDataMap);
+    }
+
+    public getCompactionQueueInfo_args() {
+    }
+
+    public getCompactionQueueInfo_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getCompactionQueueInfo_args(getCompactionQueueInfo_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);
+      }
+    }
+
+    @Override
+    public getCompactionQueueInfo_args deepCopy() {
+      return new getCompactionQueueInfo_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public getCompactionQueueInfo_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 getCompactionQueueInfo_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 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;
+
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @Override
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    @Override
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof getCompactionQueueInfo_args)
+        return this.equals((getCompactionQueueInfo_args)that);
+      return false;
+    }
+
+    public boolean equals(getCompactionQueueInfo_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetTinfo()) ? 131071 : 524287);
+      if (isSetTinfo())
+        hashCode = hashCode * 8191 + tinfo.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetCredentials()) ? 131071 : 524287);
+      if (isSetCredentials())
+        hashCode = hashCode * 8191 + credentials.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(getCompactionQueueInfo_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.compare(isSetTinfo(), other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCredentials(), other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    @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("getCompactionQueueInfo_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, java.lang.ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getCompactionQueueInfo_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getCompactionQueueInfo_argsStandardScheme getScheme() {
+        return new getCompactionQueueInfo_argsStandardScheme();
+      }
+    }
+
+    private static class getCompactionQueueInfo_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionQueueInfo_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionQueueInfo_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;
+            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, getCompactionQueueInfo_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getCompactionQueueInfo_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getCompactionQueueInfo_argsTupleScheme getScheme() {
+        return new getCompactionQueueInfo_argsTupleScheme();
+      }
+    }
+
+    private static class getCompactionQueueInfo_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionQueueInfo_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_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.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 getCompactionQueueInfo_result implements org.apache.thrift.TBase<getCompactionQueueInfo_result, getCompactionQueueInfo_result._Fields>, java.io.Serializable, Cloneable, Comparable<getCompactionQueueInfo_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getCompactionQueueInfo_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new getCompactionQueueInfo_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new getCompactionQueueInfo_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable java.util.List<TCompactionQueueSummary> 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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCompactionQueueSummary.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(getCompactionQueueInfo_result.class, metaDataMap);
+    }
+
+    public getCompactionQueueInfo_result() {
+    }
+
+    public getCompactionQueueInfo_result(
+      java.util.List<TCompactionQueueSummary> 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 getCompactionQueueInfo_result(getCompactionQueueInfo_result other) {
+      if (other.isSetSuccess()) {
+        java.util.List<TCompactionQueueSummary> __this__success = new java.util.ArrayList<TCompactionQueueSummary>(other.success.size());
+        for (TCompactionQueueSummary other_element : other.success) {
+          __this__success.add(new TCompactionQueueSummary(other_element));
+        }
+        this.success = __this__success;
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public getCompactionQueueInfo_result deepCopy() {
+      return new getCompactionQueueInfo_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.Iterator<TCompactionQueueSummary> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(TCompactionQueueSummary elem) {
+      if (this.success == null) {
+        this.success = new java.util.ArrayList<TCompactionQueueSummary>();
+      }
+      this.success.add(elem);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.util.List<TCompactionQueueSummary> getSuccess() {
+      return this.success;
+    }
+
+    public getCompactionQueueInfo_result setSuccess(@org.apache.thrift.annotation.Nullable java.util.List<TCompactionQueueSummary> 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 getCompactionQueueInfo_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.util.List<TCompactionQueueSummary>)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 getCompactionQueueInfo_result)
+        return this.equals((getCompactionQueueInfo_result)that);
+      return false;
+    }
+
+    public boolean equals(getCompactionQueueInfo_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(getCompactionQueueInfo_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("getCompactionQueueInfo_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
+    }
+
+    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 getCompactionQueueInfo_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getCompactionQueueInfo_resultStandardScheme getScheme() {
+        return new getCompactionQueueInfo_resultStandardScheme();
+      }
+    }
+
+    private static class getCompactionQueueInfo_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<getCompactionQueueInfo_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getCompactionQueueInfo_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 _list122 = iprot.readListBegin();
+                  struct.success = new java.util.ArrayList<TCompactionQueueSummary>(_list122.size);
+                  @org.apache.thrift.annotation.Nullable TCompactionQueueSummary _elem123;
+                  for (int _i124 = 0; _i124 < _list122.size; ++_i124)
+                  {
+                    _elem123 = new TCompactionQueueSummary();
+                    _elem123.read(iprot);
+                    struct.success.add(_elem123);
+                  }
+                  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 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, getCompactionQueueInfo_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 (TCompactionQueueSummary _iter125 : struct.success)
+            {
+              _iter125.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getCompactionQueueInfo_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public getCompactionQueueInfo_resultTupleScheme getScheme() {
+        return new getCompactionQueueInfo_resultTupleScheme();
+      }
+    }
+
+    private static class getCompactionQueueInfo_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<getCompactionQueueInfo_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_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.writeI32(struct.success.size());
+            for (TCompactionQueueSummary _iter126 : struct.success)
+            {
+              _iter126.write(oprot);
+            }
+          }
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getCompactionQueueInfo_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 _list127 = iprot.readListBegin(org.apache.thrift.protocol.TType.STRUCT);
+            struct.success = new java.util.ArrayList<TCompactionQueueSummary>(_list127.size);
+            @org.apache.thrift.annotation.Nullable TCompactionQueueSummary _elem128;
+            for (int _i129 = 0; _i129 < _list127.size; ++_i129)
+            {
+              _elem128 = new TCompactionQueueSummary();
+              _elem128.read(iprot);
+              struct.success.add(_elem128);
+            }
+          }
+          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 reserveCompactionJob_args implements org.apache.thrift.TBase<reserveCompactionJob_args, reserveCompactionJob_args._Fields>, java.io.Serializable, Cloneable, Comparable<reserveCompactionJob_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("reserveCompactionJob_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 QUEUE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("queueName", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField PRIORITY_FIELD_DESC = new org.apache.thrift.protocol.TField("priority", org.apache.thrift.protocol.TType.I64, (short)4);
+    private static final org.apache.thrift.protocol.TField COMPACTOR_FIELD_DESC = new org.apache.thrift.protocol.TField("compactor", org.apache.thrift.protocol.TType.STRING, (short)5);
+    private static final org.apache.thrift.protocol.TField EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", org.apache.thrift.protocol.TType.STRING, (short)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new reserveCompactionJob_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reserveCompactionJob_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 queueName; // required
+    public long priority; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String compactor; // required
+    public @org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId; // 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"),
+      QUEUE_NAME((short)3, "queueName"),
+      PRIORITY((short)4, "priority"),
+      COMPACTOR((short)5, "compactor"),
+      EXTERNAL_COMPACTION_ID((short)6, "externalCompactionId");
+
+      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: // QUEUE_NAME
+            return QUEUE_NAME;
+          case 4: // PRIORITY
+            return PRIORITY;
+          case 5: // COMPACTOR
+            return COMPACTOR;
+          case 6: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_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 __PRIORITY_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.QUEUE_NAME, new org.apache.thrift.meta_data.FieldMetaData("queueName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PRIORITY, new org.apache.thrift.meta_data.FieldMetaData("priority", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      tmpMap.put(_Fields.COMPACTOR, new org.apache.thrift.meta_data.FieldMetaData("compactor", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", 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(reserveCompactionJob_args.class, metaDataMap);
+    }
+
+    public reserveCompactionJob_args() {
+    }
+
+    public reserveCompactionJob_args(
+      org.apache.accumulo.core.clientImpl.thrift.TInfo tinfo,
+      org.apache.accumulo.core.securityImpl.thrift.TCredentials credentials,
+      java.lang.String queueName,
+      long priority,
+      java.lang.String compactor,
+      java.lang.String externalCompactionId)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.queueName = queueName;
+      this.priority = priority;
+      setPriorityIsSet(true);
+      this.compactor = compactor;
+      this.externalCompactionId = externalCompactionId;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public reserveCompactionJob_args(reserveCompactionJob_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.isSetQueueName()) {
+        this.queueName = other.queueName;
+      }
+      this.priority = other.priority;
+      if (other.isSetCompactor()) {
+        this.compactor = other.compactor;
+      }
+      if (other.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+    }
+
+    @Override
+    public reserveCompactionJob_args deepCopy() {
+      return new reserveCompactionJob_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.queueName = null;
+      setPriorityIsSet(false);
+      this.priority = 0;
+      this.compactor = null;
+      this.externalCompactionId = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public reserveCompactionJob_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 reserveCompactionJob_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 getQueueName() {
+      return this.queueName;
+    }
+
+    public reserveCompactionJob_args setQueueName(@org.apache.thrift.annotation.Nullable java.lang.String queueName) {
+      this.queueName = queueName;
+      return this;
+    }
+
+    public void unsetQueueName() {
+      this.queueName = null;
+    }
+
+    /** Returns true if field queueName is set (has been assigned a value) and false otherwise */
+    public boolean isSetQueueName() {
+      return this.queueName != null;
+    }
+
+    public void setQueueNameIsSet(boolean value) {
+      if (!value) {
+        this.queueName = null;
+      }
+    }
+
+    public long getPriority() {
+      return this.priority;
+    }
+
+    public reserveCompactionJob_args setPriority(long priority) {
+      this.priority = priority;
+      setPriorityIsSet(true);
+      return this;
+    }
+
+    public void unsetPriority() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+    }
+
+    /** Returns true if field priority is set (has been assigned a value) and false otherwise */
+    public boolean isSetPriority() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __PRIORITY_ISSET_ID);
+    }
+
+    public void setPriorityIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __PRIORITY_ISSET_ID, value);
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getCompactor() {
+      return this.compactor;
+    }
+
+    public reserveCompactionJob_args setCompactor(@org.apache.thrift.annotation.Nullable java.lang.String compactor) {
+      this.compactor = compactor;
+      return this;
+    }
+
+    public void unsetCompactor() {
+      this.compactor = null;
+    }
+
+    /** Returns true if field compactor is set (has been assigned a value) and false otherwise */
+    public boolean isSetCompactor() {
+      return this.compactor != null;
+    }
+
+    public void setCompactorIsSet(boolean value) {
+      if (!value) {
+        this.compactor = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public java.lang.String getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public reserveCompactionJob_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = 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 QUEUE_NAME:
+        if (value == null) {
+          unsetQueueName();
+        } else {
+          setQueueName((java.lang.String)value);
+        }
+        break;
+
+      case PRIORITY:
+        if (value == null) {
+          unsetPriority();
+        } else {
+          setPriority((java.lang.Long)value);
+        }
+        break;
+
+      case COMPACTOR:
+        if (value == null) {
+          unsetCompactor();
+        } else {
+          setCompactor((java.lang.String)value);
+        }
+        break;
+
+      case EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((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 QUEUE_NAME:
+        return getQueueName();
+
+      case PRIORITY:
+        return getPriority();
+
+      case COMPACTOR:
+        return getCompactor();
+
+      case EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      }
+      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 QUEUE_NAME:
+        return isSetQueueName();
+      case PRIORITY:
+        return isSetPriority();
+      case COMPACTOR:
+        return isSetCompactor();
+      case EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof reserveCompactionJob_args)
+        return this.equals((reserveCompactionJob_args)that);
+      return false;
+    }
+
+    public boolean equals(reserveCompactionJob_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_queueName = true && this.isSetQueueName();
+      boolean that_present_queueName = true && that.isSetQueueName();
+      if (this_present_queueName || that_present_queueName) {
+        if (!(this_present_queueName && that_present_queueName))
+          return false;
+        if (!this.queueName.equals(that.queueName))
+          return false;
+      }
+
+      boolean this_present_priority = true;
+      boolean that_present_priority = true;
+      if (this_present_priority || that_present_priority) {
+        if (!(this_present_priority && that_present_priority))
+          return false;
+        if (this.priority != that.priority)
+          return false;
+      }
+
+      boolean this_present_compactor = true && this.isSetCompactor();
+      boolean that_present_compactor = true && that.isSetCompactor();
+      if (this_present_compactor || that_present_compactor) {
+        if (!(this_present_compactor && that_present_compactor))
+          return false;
+        if (!this.compactor.equals(that.compactor))
+          return false;
+      }
+
+      boolean this_present_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          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 + ((isSetQueueName()) ? 131071 : 524287);
+      if (isSetQueueName())
+        hashCode = hashCode * 8191 + queueName.hashCode();
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(priority);
+
+      hashCode = hashCode * 8191 + ((isSetCompactor()) ? 131071 : 524287);
+      if (isSetCompactor())
+        hashCode = hashCode * 8191 + compactor.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(reserveCompactionJob_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(isSetQueueName(), other.isSetQueueName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetQueueName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queueName, other.queueName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetPriority(), other.isSetPriority());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetPriority()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.priority, other.priority);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetCompactor(), other.isSetCompactor());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCompactor()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compactor, other.compactor);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetExternalCompactionId(), other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        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("reserveCompactionJob_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("queueName:");
+      if (this.queueName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.queueName);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("priority:");
+      sb.append(this.priority);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("compactor:");
+      if (this.compactor == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.compactor);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      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 reserveCompactionJob_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public reserveCompactionJob_argsStandardScheme getScheme() {
+        return new reserveCompactionJob_argsStandardScheme();
+      }
+    }
+
+    private static class reserveCompactionJob_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<reserveCompactionJob_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, reserveCompactionJob_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: // QUEUE_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.queueName = iprot.readString();
+                struct.setQueueNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // PRIORITY
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.priority = iprot.readI64();
+                struct.setPriorityIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // COMPACTOR
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.compactor = iprot.readString();
+                struct.setCompactorIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(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, reserveCompactionJob_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.queueName != null) {
+          oprot.writeFieldBegin(QUEUE_NAME_FIELD_DESC);
+          oprot.writeString(struct.queueName);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(PRIORITY_FIELD_DESC);
+        oprot.writeI64(struct.priority);
+        oprot.writeFieldEnd();
+        if (struct.compactor != null) {
+          oprot.writeFieldBegin(COMPACTOR_FIELD_DESC);
+          oprot.writeString(struct.compactor);
+          oprot.writeFieldEnd();
+        }
+        if (struct.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class reserveCompactionJob_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public reserveCompactionJob_argsTupleScheme getScheme() {
+        return new reserveCompactionJob_argsTupleScheme();
+      }
+    }
+
+    private static class reserveCompactionJob_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<reserveCompactionJob_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, reserveCompactionJob_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.isSetQueueName()) {
+          optionals.set(2);
+        }
+        if (struct.isSetPriority()) {
+          optionals.set(3);
+        }
+        if (struct.isSetCompactor()) {
+          optionals.set(4);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetQueueName()) {
+          oprot.writeString(struct.queueName);
+        }
+        if (struct.isSetPriority()) {
+          oprot.writeI64(struct.priority);
+        }
+        if (struct.isSetCompactor()) {
+          oprot.writeString(struct.compactor);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, reserveCompactionJob_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.queueName = iprot.readString();
+          struct.setQueueNameIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.priority = iprot.readI64();
+          struct.setPriorityIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.compactor = iprot.readString();
+          struct.setCompactorIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(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 reserveCompactionJob_result implements org.apache.thrift.TBase<reserveCompactionJob_result, reserveCompactionJob_result._Fields>, java.io.Serializable, Cloneable, Comparable<reserveCompactionJob_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("reserveCompactionJob_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 reserveCompactionJob_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new reserveCompactionJob_resultTupleSchemeFactory();
+
+    public @org.apache.thrift.annotation.Nullable TExternalCompactionJob 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, TExternalCompactionJob.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(reserveCompactionJob_result.class, metaDataMap);
+    }
+
+    public reserveCompactionJob_result() {
+    }
+
+    public reserveCompactionJob_result(
+      TExternalCompactionJob 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 reserveCompactionJob_result(reserveCompactionJob_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TExternalCompactionJob(other.success);
+      }
+      if (other.isSetSec()) {
+        this.sec = new org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException(other.sec);
+      }
+    }
+
+    @Override
+    public reserveCompactionJob_result deepCopy() {
+      return new reserveCompactionJob_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.sec = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public TExternalCompactionJob getSuccess() {
+      return this.success;
+    }
+
+    public reserveCompactionJob_result setSuccess(@org.apache.thrift.annotation.Nullable TExternalCompactionJob 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 reserveCompactionJob_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((TExternalCompactionJob)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 reserveCompactionJob_result)
+        return this.equals((reserveCompactionJob_result)that);
+      return false;
+    }
+
+    public boolean equals(reserveCompactionJob_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(reserveCompactionJob_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("reserveCompactionJob_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 reserveCompactionJob_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public reserveCompactionJob_resultStandardScheme getScheme() {
+        return new reserveCompactionJob_resultStandardScheme();
+      }
+    }
+
+    private static class reserveCompactionJob_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<reserveCompactionJob_result> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, reserveCompactionJob_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 TExternalCompactionJob();
+                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, reserveCompactionJob_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 reserveCompactionJob_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public reserveCompactionJob_resultTupleScheme getScheme() {
+        return new reserveCompactionJob_resultTupleScheme();
+      }
+    }
+
+    private static class reserveCompactionJob_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<reserveCompactionJob_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, reserveCompactionJob_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, reserveCompactionJob_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 TExternalCompactionJob();
+          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 compactionJobFinished_args implements org.apache.thrift.TBase<compactionJobFinished_args, compactionJobFinished_args._Fields>, java.io.Serializable, Cloneable, Comparable<compactionJobFinished_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionJobFinished_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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", 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.protocol.TField FILE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("fileSize", org.apache.thrift.protocol.TType.I64, (short)5);
+    private static final org.apache.thrift.protocol.TField ENTRIES_FIELD_DESC = new org.apache.thrift.protocol.TField("entries", org.apache.thrift.protocol.TType.I64, (short)6);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new compactionJobFinished_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionJobFinished_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 externalCompactionId; // required
+    public @org.apache.thrift.annotation.Nullable org.apache.accumulo.core.dataImpl.thrift.TKeyExtent extent; // required
+    public long fileSize; // required
+    public long entries; // 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"),
+      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
+      EXTENT((short)4, "extent"),
+      FILE_SIZE((short)5, "fileSize"),
+      ENTRIES((short)6, "entries");
+
+      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: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          case 4: // EXTENT
+            return EXTENT;
+          case 5: // FILE_SIZE
+            return FILE_SIZE;
+          case 6: // ENTRIES
+            return ENTRIES;
+          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 __FILESIZE_ISSET_ID = 0;
+    private static final int __ENTRIES_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.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, 
+          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.FILE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("fileSize", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+      tmpMap.put(_Fields.ENTRIES, new org.apache.thrift.meta_data.FieldMetaData("entries", 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(compactionJobFinished_args.class, metaDataMap);
+    }
+
+    public compactionJobFinished_args() {
+    }
+
+    public compactionJobFinished_args(
+      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,
+      long fileSize,
+      long entries)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.externalCompactionId = externalCompactionId;
+      this.extent = extent;
+      this.fileSize = fileSize;
+      setFileSizeIsSet(true);
+      this.entries = entries;
+      setEntriesIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compactionJobFinished_args(compactionJobFinished_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.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+      this.fileSize = other.fileSize;
+      this.entries = other.entries;
+    }
+
+    @Override
+    public compactionJobFinished_args deepCopy() {
+      return new compactionJobFinished_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.externalCompactionId = null;
+      this.extent = null;
+      setFileSizeIsSet(false);
+      this.fileSize = 0;
+      setEntriesIsSet(false);
+      this.entries = 0;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public compactionJobFinished_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 compactionJobFinished_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 getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public compactionJobFinished_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public compactionJobFinished_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 long getFileSize() {
+      return this.fileSize;
+    }
+
+    public compactionJobFinished_args setFileSize(long fileSize) {
+      this.fileSize = fileSize;
+      setFileSizeIsSet(true);
+      return this;
+    }
+
+    public void unsetFileSize() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __FILESIZE_ISSET_ID);
+    }
+
+    /** Returns true if field fileSize is set (has been assigned a value) and false otherwise */
+    public boolean isSetFileSize() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __FILESIZE_ISSET_ID);
+    }
+
+    public void setFileSizeIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __FILESIZE_ISSET_ID, value);
+    }
+
+    public long getEntries() {
+      return this.entries;
+    }
+
+    public compactionJobFinished_args setEntries(long entries) {
+      this.entries = entries;
+      setEntriesIsSet(true);
+      return this;
+    }
+
+    public void unsetEntries() {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.clearBit(__isset_bitfield, __ENTRIES_ISSET_ID);
+    }
+
+    /** Returns true if field entries is set (has been assigned a value) and false otherwise */
+    public boolean isSetEntries() {
+      return org.apache.thrift.EncodingUtils.testBit(__isset_bitfield, __ENTRIES_ISSET_ID);
+    }
+
+    public void setEntriesIsSet(boolean value) {
+      __isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __ENTRIES_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 EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((java.lang.String)value);
+        }
+        break;
+
+      case EXTENT:
+        if (value == null) {
+          unsetExtent();
+        } else {
+          setExtent((org.apache.accumulo.core.dataImpl.thrift.TKeyExtent)value);
+        }
+        break;
+
+      case FILE_SIZE:
+        if (value == null) {
+          unsetFileSize();
+        } else {
+          setFileSize((java.lang.Long)value);
+        }
+        break;
+
+      case ENTRIES:
+        if (value == null) {
+          unsetEntries();
+        } else {
+          setEntries((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 EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      case EXTENT:
+        return getExtent();
+
+      case FILE_SIZE:
+        return getFileSize();
+
+      case ENTRIES:
+        return getEntries();
+
+      }
+      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 EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      case EXTENT:
+        return isSetExtent();
+      case FILE_SIZE:
+        return isSetFileSize();
+      case ENTRIES:
+        return isSetEntries();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof compactionJobFinished_args)
+        return this.equals((compactionJobFinished_args)that);
+      return false;
+    }
+
+    public boolean equals(compactionJobFinished_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_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          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_fileSize = true;
+      boolean that_present_fileSize = true;
+      if (this_present_fileSize || that_present_fileSize) {
+        if (!(this_present_fileSize && that_present_fileSize))
+          return false;
+        if (this.fileSize != that.fileSize)
+          return false;
+      }
+
+      boolean this_present_entries = true;
+      boolean that_present_entries = true;
+      if (this_present_entries || that_present_entries) {
+        if (!(this_present_entries && that_present_entries))
+          return false;
+        if (this.entries != that.entries)
+          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 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(fileSize);
+
+      hashCode = hashCode * 8191 + org.apache.thrift.TBaseHelper.hashCode(entries);
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compactionJobFinished_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(isSetExternalCompactionId(), other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        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(isSetFileSize(), other.isSetFileSize());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetFileSize()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileSize, other.fileSize);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = java.lang.Boolean.compare(isSetEntries(), other.isSetEntries());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetEntries()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.entries, other.entries);
+        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("compactionJobFinished_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("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      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("fileSize:");
+      sb.append(this.fileSize);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("entries:");
+      sb.append(this.entries);
+      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 compactionJobFinished_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public compactionJobFinished_argsStandardScheme getScheme() {
+        return new compactionJobFinished_argsStandardScheme();
+      }
+    }
+
+    private static class compactionJobFinished_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionJobFinished_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionJobFinished_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: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(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;
+            case 5: // FILE_SIZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.fileSize = iprot.readI64();
+                struct.setFileSizeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // ENTRIES
+              if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+                struct.entries = iprot.readI64();
+                struct.setEntriesIsSet(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, compactionJobFinished_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.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(FILE_SIZE_FIELD_DESC);
+        oprot.writeI64(struct.fileSize);
+        oprot.writeFieldEnd();
+        oprot.writeFieldBegin(ENTRIES_FIELD_DESC);
+        oprot.writeI64(struct.entries);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compactionJobFinished_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public compactionJobFinished_argsTupleScheme getScheme() {
+        return new compactionJobFinished_argsTupleScheme();
+      }
+    }
+
+    private static class compactionJobFinished_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionJobFinished_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compactionJobFinished_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.isSetExternalCompactionId()) {
+          optionals.set(2);
+        }
+        if (struct.isSetExtent()) {
+          optionals.set(3);
+        }
+        if (struct.isSetFileSize()) {
+          optionals.set(4);
+        }
+        if (struct.isSetEntries()) {
+          optionals.set(5);
+        }
+        oprot.writeBitSet(optionals, 6);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+        if (struct.isSetFileSize()) {
+          oprot.writeI64(struct.fileSize);
+        }
+        if (struct.isSetEntries()) {
+          oprot.writeI64(struct.entries);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compactionJobFinished_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.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(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.fileSize = iprot.readI64();
+          struct.setFileSizeIsSet(true);
+        }
+        if (incoming.get(5)) {
+          struct.entries = iprot.readI64();
+          struct.setEntriesIsSet(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 compactionJobFailed_args implements org.apache.thrift.TBase<compactionJobFailed_args, compactionJobFailed_args._Fields>, java.io.Serializable, Cloneable, Comparable<compactionJobFailed_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compactionJobFailed_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 EXTERNAL_COMPACTION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("externalCompactionId", 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 compactionJobFailed_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new compactionJobFailed_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 externalCompactionId; // 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"),
+      EXTERNAL_COMPACTION_ID((short)3, "externalCompactionId"),
+      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: // EXTERNAL_COMPACTION_ID
+            return EXTERNAL_COMPACTION_ID;
+          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.EXTERNAL_COMPACTION_ID, new org.apache.thrift.meta_data.FieldMetaData("externalCompactionId", 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(compactionJobFailed_args.class, metaDataMap);
+    }
+
+    public compactionJobFailed_args() {
+    }
+
+    public compactionJobFailed_args(
+      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)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.externalCompactionId = externalCompactionId;
+      this.extent = extent;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public compactionJobFailed_args(compactionJobFailed_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.isSetExternalCompactionId()) {
+        this.externalCompactionId = other.externalCompactionId;
+      }
+      if (other.isSetExtent()) {
+        this.extent = new org.apache.accumulo.core.dataImpl.thrift.TKeyExtent(other.extent);
+      }
+    }
+
+    @Override
+    public compactionJobFailed_args deepCopy() {
+      return new compactionJobFailed_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.externalCompactionId = null;
+      this.extent = null;
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.clientImpl.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public compactionJobFailed_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 compactionJobFailed_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 getExternalCompactionId() {
+      return this.externalCompactionId;
+    }
+
+    public compactionJobFailed_args setExternalCompactionId(@org.apache.thrift.annotation.Nullable java.lang.String externalCompactionId) {
+      this.externalCompactionId = externalCompactionId;
+      return this;
+    }
+
+    public void unsetExternalCompactionId() {
+      this.externalCompactionId = null;
+    }
+
+    /** Returns true if field externalCompactionId is set (has been assigned a value) and false otherwise */
+    public boolean isSetExternalCompactionId() {
+      return this.externalCompactionId != null;
+    }
+
+    public void setExternalCompactionIdIsSet(boolean value) {
+      if (!value) {
+        this.externalCompactionId = null;
+      }
+    }
+
+    @org.apache.thrift.annotation.Nullable
+    public org.apache.accumulo.core.dataImpl.thrift.TKeyExtent getExtent() {
+      return this.extent;
+    }
+
+    public compactionJobFailed_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 EXTERNAL_COMPACTION_ID:
+        if (value == null) {
+          unsetExternalCompactionId();
+        } else {
+          setExternalCompactionId((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 EXTERNAL_COMPACTION_ID:
+        return getExternalCompactionId();
+
+      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 EXTERNAL_COMPACTION_ID:
+        return isSetExternalCompactionId();
+      case EXTENT:
+        return isSetExtent();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that instanceof compactionJobFailed_args)
+        return this.equals((compactionJobFailed_args)that);
+      return false;
+    }
+
+    public boolean equals(compactionJobFailed_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_externalCompactionId = true && this.isSetExternalCompactionId();
+      boolean that_present_externalCompactionId = true && that.isSetExternalCompactionId();
+      if (this_present_externalCompactionId || that_present_externalCompactionId) {
+        if (!(this_present_externalCompactionId && that_present_externalCompactionId))
+          return false;
+        if (!this.externalCompactionId.equals(that.externalCompactionId))
+          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 + ((isSetExternalCompactionId()) ? 131071 : 524287);
+      if (isSetExternalCompactionId())
+        hashCode = hashCode * 8191 + externalCompactionId.hashCode();
+
+      hashCode = hashCode * 8191 + ((isSetExtent()) ? 131071 : 524287);
+      if (isSetExtent())
+        hashCode = hashCode * 8191 + extent.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(compactionJobFailed_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(isSetExternalCompactionId(), other.isSetExternalCompactionId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetExternalCompactionId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.externalCompactionId, other.externalCompactionId);
+        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("compactionJobFailed_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("externalCompactionId:");
+      if (this.externalCompactionId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.externalCompactionId);
+      }
+      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 compactionJobFailed_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public compactionJobFailed_argsStandardScheme getScheme() {
+        return new compactionJobFailed_argsStandardScheme();
+      }
+    }
+
+    private static class compactionJobFailed_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<compactionJobFailed_args> {
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol iprot, compactionJobFailed_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: // EXTERNAL_COMPACTION_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.externalCompactionId = iprot.readString();
+                struct.setExternalCompactionIdIsSet(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, compactionJobFailed_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.externalCompactionId != null) {
+          oprot.writeFieldBegin(EXTERNAL_COMPACTION_ID_FIELD_DESC);
+          oprot.writeString(struct.externalCompactionId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.extent != null) {
+          oprot.writeFieldBegin(EXTENT_FIELD_DESC);
+          struct.extent.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class compactionJobFailed_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      @Override
+      public compactionJobFailed_argsTupleScheme getScheme() {
+        return new compactionJobFailed_argsTupleScheme();
+      }
+    }
+
+    private static class compactionJobFailed_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<compactionJobFailed_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, compactionJobFailed_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.isSetExternalCompactionId()) {
+          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.isSetExternalCompactionId()) {
+          oprot.writeString(struct.externalCompactionId);
+        }
+        if (struct.isSetExtent()) {
+          struct.extent.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, compactionJobFailed_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.externalCompactionId = iprot.readString();
+          struct.setExternalCompactionIdIsSet(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/client.thrift b/core/src/main/thrift/client.thrift
index 77e9324..8891195 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -20,7 +20,6 @@
 namespace cpp org.apache.accumulo.core.clientImpl.thrift
 
 include "security.thrift"
-include "trace.thrift"
 
 enum TableOperation {
   CREATE
@@ -47,13 +46,13 @@
   NOTFOUND
   OFFLINE
   BULK_BAD_INPUT_DIRECTORY
-  BULK_BAD_ERROR_DIRECTORY
+  OBSOLETE_BULK_BAD_ERROR_DIRECTORY
   BAD_RANGE
   OTHER
   NAMESPACE_EXISTS
   NAMESPACE_NOTFOUND
   INVALID_NAME
-  BULK_BAD_LOAD_MAPPING
+  OBSOLETE_BULK_BAD_LOAD_MAPPING
   BULK_CONCURRENT_MERGE
 }
 
@@ -117,6 +116,10 @@
    2:map<string, string> properties
 }
 
+struct TInfo {
+  1:map<string,string> headers
+}
+
 service ClientService {
 
   // system management methods
@@ -124,23 +127,9 @@
   string getInstanceId()
   string getZooKeepers()
 
-  // deprecated for new bulkImport
-  list<string> bulkImportFiles(
-    1:trace.TInfo tinfo
-    8:security.TCredentials credentials
-    3:i64 tid
-    4:string tableId
-    5:list<string> files
-    6:string errorDir
-    7:bool setTime
-  ) throws (
-    1:ThriftSecurityException sec
-    2:ThriftTableOperationException tope
-  )
-
   // ensures that nobody is working on the transaction id above
   bool isActive(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:i64 tid
   )
 
@@ -160,14 +149,14 @@
 
   // user management methods
   set<string> listLocalUsers(
-    2:trace.TInfo tinfo
+    2:TInfo tinfo
     3:security.TCredentials credentials
   ) throws (
     1:ThriftSecurityException sec
   )
 
   void createLocalUser(
-    5:trace.TInfo tinfo
+    5:TInfo tinfo
     6:security.TCredentials credentials
     2:string principal
     3:binary password
@@ -176,7 +165,7 @@
   )
 
   void dropLocalUser(
-    3:trace.TInfo tinfo
+    3:TInfo tinfo
     4:security.TCredentials credentials
     2:string principal
   ) throws (
@@ -184,7 +173,7 @@
   )
 
   void changeLocalUserPassword(
-    4:trace.TInfo tinfo
+    4:TInfo tinfo
     5:security.TCredentials credentials
     2:string principal
     3:binary password
@@ -194,14 +183,14 @@
 
   // authentication-related methods
   bool authenticate(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:ThriftSecurityException sec
   )
 
   bool authenticateUser(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:security.TCredentials toAuth
   ) throws (
@@ -210,7 +199,7 @@
 
   // authorization-related methods
   void changeAuthorizations(
-    4:trace.TInfo tinfo
+    4:TInfo tinfo
     5:security.TCredentials credentials
     2:string principal
     3:list<binary> authorizations
@@ -219,7 +208,7 @@
   )
 
   list<binary> getUserAuthorizations(
-    3:trace.TInfo tinfo
+    3:TInfo tinfo
     4:security.TCredentials credentials
     2:string principal
   ) throws (
@@ -228,7 +217,7 @@
 
   // permissions-related methods
   bool hasSystemPermission(
-    4:trace.TInfo tinfo
+    4:TInfo tinfo
     5:security.TCredentials credentials
     2:string principal
     3:i8 sysPerm
@@ -237,7 +226,7 @@
   )
 
   bool hasTablePermission(
-    5:trace.TInfo tinfo
+    5:TInfo tinfo
     6:security.TCredentials credentials
     2:string principal
     3:string tableName
@@ -248,7 +237,7 @@
   )
 
   bool hasNamespacePermission(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string principal
     4:string ns
@@ -259,7 +248,7 @@
   )
 
   void grantSystemPermission(
-    4:trace.TInfo tinfo
+    4:TInfo tinfo
     5:security.TCredentials credentials
     2:string principal
     3:i8 permission
@@ -268,7 +257,7 @@
   )
 
   void revokeSystemPermission(
-    4:trace.TInfo tinfo
+    4:TInfo tinfo
     5:security.TCredentials credentials
     2:string principal
     3:i8 permission
@@ -277,7 +266,7 @@
   )
 
   void grantTablePermission(
-    5:trace.TInfo tinfo
+    5:TInfo tinfo
     6:security.TCredentials credentials
     2:string principal
     3:string tableName
@@ -288,7 +277,7 @@
   )
 
   void revokeTablePermission(
-    5:trace.TInfo tinfo
+    5:TInfo tinfo
     6:security.TCredentials credentials
     2:string principal
     3:string tableName
@@ -299,7 +288,7 @@
   )
 
   void grantNamespacePermission(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string principal
     4:string ns
@@ -310,7 +299,7 @@
   )
 
   void revokeNamespacePermission(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string principal
     4:string ns
@@ -322,7 +311,7 @@
 
   // configuration methods
   map<string, string> getConfiguration(
-    2:trace.TInfo tinfo
+    2:TInfo tinfo
     3:security.TCredentials credentials
     1:ConfigurationType type
   ) throws (
@@ -330,21 +319,21 @@
   )
 
   map<string, string> getSystemProperties(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:ThriftSecurityException sec
   )
 
   TVersionedProperties getVersionedSystemProperties(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:ThriftSecurityException sec
   )
 
   map<string, string> getTableConfiguration(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     3:security.TCredentials credentials
     2:string tableName
   ) throws (
@@ -353,7 +342,7 @@
   )
 
   map<string, string> getTableProperties(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     3:security.TCredentials credentials
     2:string tableName
   ) throws (
@@ -362,7 +351,7 @@
   )
 
   TVersionedProperties getVersionedTableProperties(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     3:security.TCredentials credentials
     2:string tableName
   ) throws (
@@ -371,7 +360,7 @@
   )
 
   map<string, string> getNamespaceConfiguration(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string ns
   ) throws (
@@ -380,7 +369,7 @@
   )
 
   map<string, string> getNamespaceProperties(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string ns
   ) throws (
@@ -389,7 +378,7 @@
   )
 
   TVersionedProperties getVersionedNamespaceProperties(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string ns
   ) throws (
@@ -398,14 +387,14 @@
   )
 
   bool checkClass(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     4:security.TCredentials credentials
     2:string className
     3:string interfaceMatch
   )
 
   bool checkTableClass(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     5:security.TCredentials credentials
     2:string tableId
     3:string className
@@ -416,7 +405,7 @@
   )
 
   bool checkNamespaceClass(
-    1:trace.TInfo tinfo
+    1:TInfo tinfo
     2:security.TCredentials credentials
     3:string namespaceId
     4:string className
diff --git a/core/src/main/thrift/compaction-coordinator.thrift b/core/src/main/thrift/compaction-coordinator.thrift
index 7cb090b..245cdc6 100644
--- a/core/src/main/thrift/compaction-coordinator.thrift
+++ b/core/src/main/thrift/compaction-coordinator.thrift
@@ -23,7 +23,6 @@
 include "data.thrift"
 include "security.thrift"
 include "tabletserver.thrift"
-include "trace.thrift"
 
 enum TCompactionState {
   # Coordinator should set state to ASSIGNED when getCompactionJob is called by Compactor
@@ -67,7 +66,7 @@
    * Called by Compactor on successful completion of compaction job
    */
   void compactionCompleted(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials  
     3:string externalCompactionId
     4:data.TKeyExtent extent
@@ -78,7 +77,7 @@
    * Called by Compactor to get the next compaction job
    */
   tabletserver.TExternalCompactionJob getCompactionJob(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string queueName
     4:string compactor
@@ -89,7 +88,7 @@
    * Called by Compactor to update the Coordinator with the state of the compaction
    */
   void updateCompactionStatus(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string externalCompactionId
     4:TCompactionStatusUpdate status
@@ -100,7 +99,7 @@
    * Called by Compactor on unsuccessful completion of compaction job
    */
   void compactionFailed(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string externalCompactionId
     4:data.TKeyExtent extent
@@ -110,7 +109,7 @@
    * Called by the Monitor to get progress information
    */
   TExternalCompactionList getRunningCompactions(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   )
 
@@ -118,12 +117,12 @@
    * Called by the Monitor to get progress information
    */
   TExternalCompactionList getCompletedCompactions(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   )
 
   void cancel(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string externalCompactionId
   )
@@ -133,28 +132,28 @@
 service CompactorService {
 
   tabletserver.TExternalCompactionJob getRunningCompaction(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
 
   string getRunningCompactionId(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
 
   list<tabletserver.ActiveCompaction> getActiveCompactions(
-    2:trace.TInfo tinfo
+    2:client.TInfo tinfo
     1:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
 
   void cancel(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string externalCompactionId
   )
diff --git a/core/src/main/thrift/data.thrift b/core/src/main/thrift/data.thrift
index d2ec701..d83c99f 100644
--- a/core/src/main/thrift/data.thrift
+++ b/core/src/main/thrift/data.thrift
@@ -125,10 +125,6 @@
   2:TCMStatus status
 }
 
-struct MapFileInfo {
-  1:i64 estimatedSize
-}
-
 struct TCondition {
   1:binary cf
   2:binary cq
@@ -189,5 +185,3 @@
 
 typedef map<TKeyExtent, list<TMutation>> UpdateBatch
 
-typedef map<TKeyExtent, map<string, MapFileInfo>> TabletFiles
-
diff --git a/core/src/main/thrift/gc.thrift b/core/src/main/thrift/gc.thrift
index 8e5232d..a4a2ba1 100644
--- a/core/src/main/thrift/gc.thrift
+++ b/core/src/main/thrift/gc.thrift
@@ -20,7 +20,6 @@
 namespace cpp org.apache.accumulo.core.gc.thrift
 
 include "security.thrift"
-include "trace.thrift"
 include "client.thrift"
 
 struct GcCycleStats {
@@ -43,7 +42,7 @@
 service GCMonitorService {
 
   GCStatus getStatus(
-    2:trace.TInfo tinfo
+    2:client.TInfo tinfo
     1:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
diff --git a/core/src/main/thrift/manager.thrift b/core/src/main/thrift/manager.thrift
index b601bca..a334e0e 100644
--- a/core/src/main/thrift/manager.thrift
+++ b/core/src/main/thrift/manager.thrift
@@ -22,8 +22,6 @@
 include "data.thrift"
 include "security.thrift"
 include "client.thrift"
-include "trace.thrift"
-include "master.thrift"
 
 struct DeadServer {
   1:string server
@@ -42,17 +40,12 @@
   3:string description
 }
 
-exception RecoveryException {
-  1:string why
-}
-
 enum TabletLoadState {
   LOADED
   LOAD_FAILURE
   UNLOADED
   UNLOAD_FAILURE_NOT_SERVING
   UNLOAD_ERROR
-  CHOPPED
 }
 
 enum FateOperation {
@@ -64,7 +57,7 @@
   TABLE_OFFLINE
   TABLE_MERGE
   TABLE_DELETE_RANGE
-  TABLE_BULK_IMPORT
+  OBSOLETE_TABLE_BULK_IMPORT
   TABLE_COMPACT
   TABLE_IMPORT
   TABLE_EXPORT
@@ -91,23 +84,91 @@
   NORMAL
 }
 
+struct Compacting {
+  1:i32 running
+  2:i32 queued
+}
+
+struct TableInfo {
+  1:i64 recs
+  2:i64 recsInMemory
+  3:i32 tablets
+  4:i32 onlineTablets
+  5:double ingestRate
+  6:double ingestByteRate
+  7:double queryRate
+  8:double queryByteRate
+  9:Compacting minors
+  10:Compacting majors
+  11:Compacting scans
+  12:double scanRate
+}
+
+struct RecoveryStatus {
+  2:string name
+  // in millis
+  5:i32 runtime
+  6:double progress
+}
+
+enum BulkImportState {
+  INITIAL
+  // manager moves the files into the accumulo area
+  MOVING
+  // tserver examines the index of the file
+  PROCESSING
+  // tserver assigns the file to tablets
+  ASSIGNING
+  // tserver incorporates file into tablet
+  LOADING
+  // manager moves error files into the error directory
+  COPY_FILES
+  // flags and locks removed
+  CLEANUP
+}
+
+struct BulkImportStatus {
+  1:i64 startTime
+  2:string filename
+  3:BulkImportState state
+}
+
+struct TabletServerStatus {
+  1:map<string, TableInfo> tableMap
+  2:i64 lastContact
+  3:string name
+  5:double osLoad
+  7:i64 holdTime
+  8:i64 lookups
+  10:i64 indexCacheHits
+  11:i64 indexCacheRequest
+  12:i64 dataCacheHits
+  13:i64 dataCacheRequest
+  14:list<RecoveryStatus> logSorts
+  15:i64 flushs
+  16:i64 syncs
+  17:list<BulkImportStatus> bulkImports
+  19:string version
+  18:i64 responseTime
+}
+
 struct ManagerMonitorInfo {
-  1:map<string, master.TableInfo> tableMap
-  2:list<master.TabletServerStatus> tServerInfo
+  1:map<string, TableInfo> tableMap
+  2:list<TabletServerStatus> tServerInfo
   3:map<string, i8> badTServers
   4:ManagerState state
   5:ManagerGoalState goalState
   6:i32 unassignedTablets
   7:set<string> serversShuttingDown
   8:list<DeadServer> deadTabletServers
-  9:list<master.BulkImportStatus> bulkImports
+  9:list<BulkImportStatus> bulkImports
 }
 
 service FateService {
 
   // register a fate operation by reserving an opid
   i64 beginFateOperation(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
@@ -116,7 +177,7 @@
 
   // initiate execution of the fate operation; set autoClean to true if not waiting for completion
   void executeFateOperation(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:i64 opid
     4:FateOperation op
@@ -131,7 +192,7 @@
 
   // wait for completion of the operation and get the returned exception, if any
   string waitForFateOperation(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:i64 opid
   ) throws (
@@ -142,7 +203,7 @@
 
   // clean up fate operation if autoClean was not set, after waiting
   void finishFateOperation(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:i64 opid
   ) throws (
@@ -152,7 +213,7 @@
 
   // cancel a fate operation
   bool cancelFateOperation(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:i64 opid
   ) throws (
@@ -166,7 +227,7 @@
 
   // table management methods
   i64 initiateFlush(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string tableName
   ) throws (
@@ -176,7 +237,7 @@
   )
 
   void waitForFlush(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string tableName
     4:binary startRow
@@ -190,7 +251,7 @@
   )
 
   void setTableProperty(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string tableName
     4:string property
@@ -203,7 +264,7 @@
   )
 
   void modifyTableProperties(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string tableName
     4:client.TVersionedProperties vProperties
@@ -216,7 +277,7 @@
   )
 
   void removeTableProperty(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string tableName
     4:string property
@@ -227,7 +288,7 @@
   )
 
   void setNamespaceProperty(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string ns
     4:string property
@@ -240,7 +301,7 @@
   )
 
   void modifyNamespaceProperties(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string ns
     4:client.TVersionedProperties vProperties
@@ -252,7 +313,7 @@
   )
 
   void removeNamespaceProperty(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string ns
     4:string property
@@ -264,7 +325,7 @@
 
   // system management methods
   void setManagerGoalState(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:ManagerGoalState state
   ) throws (
@@ -273,7 +334,7 @@
   )
 
   void shutdown(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:bool stopTabletServers
   ) throws (
@@ -282,7 +343,7 @@
   )
 
   void shutdownTabletServer(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string tabletServer
     4:bool force
@@ -292,7 +353,7 @@
   )
 
   void setSystemProperty(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string property
     4:string value
@@ -303,7 +364,7 @@
   )
  
   void modifySystemProperties(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:client.TVersionedProperties vProperties
   ) throws (
@@ -314,7 +375,7 @@
   )
 
   void removeSystemProperty(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string property
   ) throws (
@@ -324,7 +385,7 @@
 
   // system monitoring methods
   ManagerMonitorInfo getManagerStats(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
@@ -332,21 +393,21 @@
   )
 
   void waitForBalance(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
   ) throws (
     1:client.ThriftNotActiveServiceException tnase
   )
 
   // tablet server reporting
   oneway void reportSplitExtent(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string serverName
     4:TabletSplit split
   )
 
   oneway void reportTabletStatus(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string serverName
     4:TabletLoadState status
@@ -354,7 +415,7 @@
   )
 
   list<string> getActiveTservers(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
@@ -363,7 +424,7 @@
 
   // Delegation token request
   security.TDelegationToken getDelegationToken(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:security.TDelegationTokenConfig cfg
   ) throws (
@@ -371,14 +432,4 @@
     2:client.ThriftNotActiveServiceException tnase
   )
 
-  // Determine when all provided logs are replicated
-  bool drainReplicationTable(
-    1:trace.TInfo tfino
-    2:security.TCredentials credentials
-    3:string tableName
-    4:set<string> logsToWatch
-  ) throws (
-    1:client.ThriftNotActiveServiceException tnase
-  )
-
 }
diff --git a/core/src/main/thrift/master.thrift b/core/src/main/thrift/master.thrift
deleted file mode 100644
index 9da7ba8..0000000
--- a/core/src/main/thrift/master.thrift
+++ /dev/null
@@ -1,93 +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.
- */
-namespace java org.apache.accumulo.core.master.thrift
-namespace cpp org.apache.accumulo.core.master.thrift
-
-include "data.thrift"
-include "security.thrift"
-include "client.thrift"
-include "trace.thrift"
-
-struct Compacting {
-  1:i32 running
-  2:i32 queued
-}
-
-struct TableInfo {
-  1:i64 recs
-  2:i64 recsInMemory
-  3:i32 tablets
-  4:i32 onlineTablets
-  5:double ingestRate
-  6:double ingestByteRate
-  7:double queryRate
-  8:double queryByteRate
-  9:Compacting minors
-  10:Compacting majors
-  11:Compacting scans
-  12:double scanRate
-}
-
-struct RecoveryStatus {
-  2:string name
-  // in millis
-  5:i32 runtime
-  6:double progress
-}
-
-enum BulkImportState {
-  INITIAL
-  // manager moves the files into the accumulo area
-  MOVING
-  // tserver examines the index of the file
-  PROCESSING
-  // tserver assigns the file to tablets
-  ASSIGNING
-  // tserver incorporates file into tablet
-  LOADING
-  // manager moves error files into the error directory
-  COPY_FILES
-  // flags and locks removed
-  CLEANUP
-}
-
-struct BulkImportStatus {
-  1:i64 startTime
-  2:string filename
-  3:BulkImportState state
-}
-
-struct TabletServerStatus {
-  1:map<string, TableInfo> tableMap
-  2:i64 lastContact
-  3:string name
-  5:double osLoad
-  7:i64 holdTime
-  8:i64 lookups
-  10:i64 indexCacheHits
-  11:i64 indexCacheRequest
-  12:i64 dataCacheHits
-  13:i64 dataCacheRequest
-  14:list<RecoveryStatus> logSorts
-  15:i64 flushs
-  16:i64 syncs
-  17:list<BulkImportStatus> bulkImports
-  19:string version
-  18:i64 responseTime
-}
diff --git a/core/src/main/thrift/replication.thrift b/core/src/main/thrift/replication.thrift
deleted file mode 100644
index 155227c..0000000
--- a/core/src/main/thrift/replication.thrift
+++ /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.
- */
-namespace java org.apache.accumulo.core.replication.thrift
-namespace cpp org.apache.accumulo.core.replication.thrift
-
-include "data.thrift"
-include "security.thrift"
-
-struct WalEdits {
-  1:list<binary> edits
-}
-
-struct KeyValues {
-  1:list<data.TKeyValue> keyValues
-}
-
-enum RemoteReplicationErrorCode {
-  COULD_NOT_DESERIALIZE
-  COULD_NOT_APPLY
-  TABLE_DOES_NOT_EXIST
-  CANNOT_AUTHENTICATE
-  CANNOT_INSTANTIATE_REPLAYER
-}
-
-enum ReplicationCoordinatorErrorCode {
-  NO_AVAILABLE_SERVERS
-  SERVICE_CONFIGURATION_UNAVAILABLE
-  CANNOT_AUTHENTICATE
-}
-
-exception ReplicationCoordinatorException {
-  1:ReplicationCoordinatorErrorCode code
-  2:string reason
-}
-
-exception RemoteReplicationException {
-  1:RemoteReplicationErrorCode code
-  2:string reason
-}
-
-service ReplicationCoordinator {
-
-  string getServicerAddress(
-    1:string remoteTableId
-    2:security.TCredentials credentials
-  ) throws (
-    1:ReplicationCoordinatorException e
-  )
-
-}
-
-service ReplicationServicer {
-
-  i64 replicateLog(
-    1:string remoteTableId
-    2:WalEdits data
-    3:security.TCredentials credentials
-  ) throws (
-    1:RemoteReplicationException e
-  )
-
-  i64 replicateKeyValues(
-    1:string remoteTableId
-    2:KeyValues data
-    3:security.TCredentials credentials
-  ) throws (
-    1:RemoteReplicationException e
-  )
-
-}
diff --git a/core/src/main/thrift/tabletingest.thrift b/core/src/main/thrift/tabletingest.thrift
new file mode 100644
index 0000000..5498ce1
--- /dev/null
+++ b/core/src/main/thrift/tabletingest.thrift
@@ -0,0 +1,113 @@
+/*
+ * 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.
+ */
+namespace java org.apache.accumulo.core.tabletingest.thrift
+namespace cpp org.apache.accumulo.core.tabletingest.thrift
+
+include "data.thrift"
+include "security.thrift"
+include "client.thrift"
+include "manager.thrift"
+include "tabletserver.thrift"
+
+exception ConstraintViolationException {
+  1:list<data.TConstraintViolationSummary> violationSummaries
+}
+
+enum TDurability {
+  DEFAULT = 0
+  SYNC = 1
+  FLUSH = 2
+  LOG = 3
+  NONE = 4
+}
+
+struct DataFileInfo {
+  1:i64 estimatedSize
+}
+
+service TabletIngestClientService {
+
+  //the following calls support a batch update to multiple tablets on a tablet server
+  data.UpdateID startUpdate(
+    2:client.TInfo tinfo
+    1:security.TCredentials credentials
+    3:TDurability durability
+  ) throws (
+    1:client.ThriftSecurityException sec
+  )
+
+  oneway void applyUpdates(
+    1:client.TInfo tinfo
+    2:data.UpdateID updateID
+    3:data.TKeyExtent keyExtent
+    4:list<data.TMutation> mutations
+  )
+
+  data.UpdateErrors closeUpdate(
+    2:client.TInfo tinfo
+    1:data.UpdateID updateID
+  ) throws (
+    1:tabletserver.NoSuchScanIDException nssi
+  )
+
+  bool cancelUpdate(
+    1:client.TInfo tinfo
+    2:data.UpdateID updateID
+  )
+
+  data.TConditionalSession startConditionalUpdate(
+    1:client.TInfo tinfo
+    2:security.TCredentials credentials
+    3:list<binary> authorizations
+    4:string tableID
+    5:TDurability durability
+    6:string classLoaderContext
+  ) throws (
+    1:client.ThriftSecurityException sec
+  )
+
+  list<data.TCMResult> conditionalUpdate(
+    1:client.TInfo tinfo
+    2:data.UpdateID sessID
+    3:data.CMBatch mutations
+    4:list<string> symbols
+  ) throws (
+    1:tabletserver.NoSuchScanIDException nssi
+  )
+
+  void invalidateConditionalUpdate(
+    1:client.TInfo tinfo
+    2:data.UpdateID sessID
+  )
+
+  oneway void closeConditionalUpdate(
+    1:client.TInfo tinfo
+    2:data.UpdateID sessID
+  )
+
+  oneway void loadFiles(
+    1:client.TInfo tinfo
+    2:security.TCredentials credentials
+    3:i64 tid
+    4:string dir
+    7:map<data.TKeyExtent, map<string, DataFileInfo>> files
+    6:bool setTime
+  )
+
+}
diff --git a/core/src/main/thrift/tabletmgmt.thrift b/core/src/main/thrift/tabletmgmt.thrift
new file mode 100644
index 0000000..4a3d3e0
--- /dev/null
+++ b/core/src/main/thrift/tabletmgmt.thrift
@@ -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.
+ */
+namespace java org.apache.accumulo.core.tablet.thrift
+namespace cpp org.apache.accumulo.core.tablet.thrift
+
+include "data.thrift"
+include "security.thrift"
+include "client.thrift"
+include "tabletserver.thrift"
+
+enum TUnloadTabletGoal {
+  UNKNOWN
+  UNASSIGNED
+  SUSPENDED
+  DELETED
+}
+
+service TabletManagementClientService {
+
+  oneway void loadTablet(
+    5:client.TInfo tinfo
+    1:security.TCredentials credentials
+    4:string lock
+    2:data.TKeyExtent extent
+  )
+
+  oneway void unloadTablet(
+    5:client.TInfo tinfo
+    1:security.TCredentials credentials
+    4:string lock
+    2:data.TKeyExtent extent
+    6:TUnloadTabletGoal goal
+    7:i64 requestTime
+  )
+
+  void splitTablet(
+    4:client.TInfo tinfo
+    1:security.TCredentials credentials
+    2:data.TKeyExtent extent
+    3:binary splitPoint
+  ) throws (
+    1:client.ThriftSecurityException sec
+    2:tabletserver.NotServingTabletException nste
+  )
+
+  oneway void flushTablet(
+    1:client.TInfo tinfo
+    2:security.TCredentials credentials
+    3:string lock
+    4:data.TKeyExtent extent
+  )
+
+}
diff --git a/core/src/main/thrift/tabletscan.thrift b/core/src/main/thrift/tabletscan.thrift
new file mode 100644
index 0000000..5c770a7
--- /dev/null
+++ b/core/src/main/thrift/tabletscan.thrift
@@ -0,0 +1,167 @@
+/*
+ * 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.
+ */
+namespace java org.apache.accumulo.core.tabletscan.thrift
+namespace cpp org.apache.accumulo.core.tabletscan.thrift
+
+include "data.thrift"
+include "security.thrift"
+include "client.thrift"
+include "manager.thrift"
+include "tabletserver.thrift"
+
+exception TooManyFilesException {
+  1:data.TKeyExtent extent
+}
+
+exception TSampleNotPresentException {
+  1:data.TKeyExtent extent
+}
+
+exception ScanServerBusyException {}
+
+struct TSamplerConfiguration {
+  1:string className
+  2:map<string, string> options
+}
+
+enum ScanType {
+  SINGLE
+  BATCH
+}
+
+enum ScanState {
+  IDLE
+  RUNNING
+  QUEUED
+}
+
+struct ActiveScan {
+  2:string client
+  3:string user
+  4:string tableId
+  5:i64 age
+  6:i64 idleTime
+  7:ScanType type
+  8:ScanState state
+  9:data.TKeyExtent extent
+  10:list<data.TColumn> columns
+  11:list<data.IterInfo> ssiList
+  // Server Side Iterator Options
+  12:map<string, map<string, string>> ssio
+  13:list<binary> authorizations
+  14:optional i64 scanId
+  // name of the classloader context
+  15:string classLoaderContext
+}
+
+
+service TabletScanClientService {
+
+  // scan a range of keys
+  data.InitialScan startScan(
+    11:client.TInfo tinfo
+    1:security.TCredentials credentials
+    2:data.TKeyExtent extent
+    3:data.TRange range
+    4:list<data.TColumn> columns
+    5:i32 batchSize
+    6:list<data.IterInfo> ssiList
+    7:map<string, map<string, string>> ssio
+    8:list<binary> authorizations
+    9:bool waitForWrites
+    10:bool isolated
+    12:i64 readaheadThreshold
+    13:TSamplerConfiguration samplerConfig
+    14:i64 batchTimeOut
+    // name of the classloader context
+    15:string classLoaderContext
+    16:map<string, string> executionHints
+    17:i64 busyTimeout
+  ) throws (
+    1:client.ThriftSecurityException sec
+    2:tabletserver.NotServingTabletException nste
+    3:TooManyFilesException tmfe
+    4:TSampleNotPresentException tsnpe
+    5:ScanServerBusyException ssbe
+  )
+
+  data.ScanResult continueScan(
+    2:client.TInfo tinfo
+    1:data.ScanID scanID
+    3:i64 busyTimeout
+  ) throws (
+    1:tabletserver.NoSuchScanIDException nssi
+    2:tabletserver.NotServingTabletException nste
+    3:TooManyFilesException tmfe
+    4:TSampleNotPresentException tsnpe
+    5:ScanServerBusyException ssbe
+  )
+
+  oneway void closeScan(
+    2:client.TInfo tinfo
+    1:data.ScanID scanID
+  )
+
+  // scan over a series of ranges
+  data.InitialMultiScan startMultiScan(
+    8:client.TInfo tinfo
+    1:security.TCredentials credentials
+    2:data.ScanBatch batch
+    3:list<data.TColumn> columns
+    4:list<data.IterInfo> ssiList
+    5:map<string, map<string, string>> ssio
+    6:list<binary> authorizations
+    7:bool waitForWrites
+    9:TSamplerConfiguration samplerConfig
+    10:i64 batchTimeOut
+    // name of the classloader context
+    11:string classLoaderContext
+    12:map<string, string> executionHints
+    13:i64 busyTimeout
+  ) throws (
+    1:client.ThriftSecurityException sec
+    2:TSampleNotPresentException tsnpe
+    3:ScanServerBusyException ssbe
+  )
+
+  data.MultiScanResult continueMultiScan(
+    2:client.TInfo tinfo
+    1:data.ScanID scanID
+    3:i64 busyTimeout
+  ) throws (
+    1:tabletserver.NoSuchScanIDException nssi
+    2:TSampleNotPresentException tsnpe
+    3:ScanServerBusyException ssbe
+  )
+
+  void closeMultiScan(
+    2:client.TInfo tinfo
+    1:data.ScanID scanID
+  ) throws (
+    1:tabletserver.NoSuchScanIDException nssi
+  )
+
+  list<ActiveScan> getActiveScans(
+    2:client.TInfo tinfo
+    1:security.TCredentials credentials
+  ) throws (
+    1:client.ThriftSecurityException sec
+  )
+
+}
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index 9fdc76b..cb7c724 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -23,29 +23,13 @@
 include "security.thrift"
 include "client.thrift"
 include "manager.thrift"
-include "master.thrift"
-include "trace.thrift"
 
 exception NotServingTabletException {
   1:data.TKeyExtent extent
 }
 
-exception TooManyFilesException {
-  1:data.TKeyExtent extent
-}
-
-exception TSampleNotPresentException {
-  1:data.TKeyExtent extent
-}
-
 exception NoSuchScanIDException {}
 
-exception ScanServerBusyException {}
-
-exception ConstraintViolationException {
-  1:list<data.TConstraintViolationSummary> violationSummaries
-}
-
 struct ActionStats {
   1:i32 status
   2:double elapsed
@@ -71,36 +55,6 @@
   8:i64 splitCreationTime
 }
 
-enum ScanType {
-  SINGLE
-  BATCH
-}
-
-enum ScanState {
-  IDLE
-  RUNNING
-  QUEUED
-}
-
-struct ActiveScan {
-  2:string client
-  3:string user
-  4:string tableId
-  5:i64 age
-  6:i64 idleTime
-  7:ScanType type
-  8:ScanState state
-  9:data.TKeyExtent extent
-  10:list<data.TColumn> columns
-  11:list<data.IterInfo> ssiList
-  // Server Side Iterator Options
-  12:map<string, map<string, string>> ssio
-  13:list<binary> authorizations
-  14:optional i64 scanId
-  // name of the classloader context
-  15:string classLoaderContext
-}
-
 enum TCompactionType {
   MINOR
   MERGE
@@ -111,19 +65,10 @@
 enum TCompactionReason {
   USER
   SYSTEM
-  CHOP
   IDLE
   CLOSE
 }
 
-enum TDurability {
-  DEFAULT = 0
-  SYNC = 1
-  FLUSH = 2
-  LOG = 3
-  NONE = 4
-}
-
 struct ActiveCompaction {
   1:data.TKeyExtent extent
   2:i64 age
@@ -136,6 +81,7 @@
   9:i64 entriesWritten
   10:list<data.IterInfo> ssiList
   11:map<string, map<string, string>> ssio
+  12:i64 timesPaused
 }
 
 struct TIteratorSetting {
@@ -149,18 +95,6 @@
   1:list<TIteratorSetting> iterators
 }
 
-struct TSamplerConfiguration {
-  1:string className
-  2:map<string, string> options
-}
-
-enum TUnloadTabletGoal {
-  UNKNOWN
-  UNASSIGNED
-  SUSPENDED
-  DELETED
-}
-
 struct InputFile {
   1:string metadataFileEntry
   2:i64 size
@@ -181,7 +115,6 @@
 }
 
 enum TCompactionKind {
-  CHOP
   SELECTOR
   SYSTEM
   USER
@@ -198,222 +131,10 @@
   3:i64 fileSize;
 }
 
-service TabletScanClientService {
-
-  // scan a range of keys
-  data.InitialScan startScan(
-    11:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    2:data.TKeyExtent extent
-    3:data.TRange range
-    4:list<data.TColumn> columns
-    5:i32 batchSize
-    6:list<data.IterInfo> ssiList
-    7:map<string, map<string, string>> ssio
-    8:list<binary> authorizations
-    9:bool waitForWrites
-    10:bool isolated
-    12:i64 readaheadThreshold
-    13:TSamplerConfiguration samplerConfig
-    14:i64 batchTimeOut
-    // name of the classloader context
-    15:string classLoaderContext
-    16:map<string, string> executionHints
-    17:i64 busyTimeout
-  ) throws (
-    1:client.ThriftSecurityException sec
-    2:NotServingTabletException nste
-    3:TooManyFilesException tmfe
-    4:TSampleNotPresentException tsnpe
-    5:ScanServerBusyException ssbe
-  )
-
-  data.ScanResult continueScan(
-    2:trace.TInfo tinfo
-    1:data.ScanID scanID
-    3:i64 busyTimeout
-  ) throws (
-    1:NoSuchScanIDException nssi
-    2:NotServingTabletException nste
-    3:TooManyFilesException tmfe
-    4:TSampleNotPresentException tsnpe
-    5:ScanServerBusyException ssbe
-  )
-
-  oneway void closeScan(
-    2:trace.TInfo tinfo
-    1:data.ScanID scanID
-  )
-
-  // scan over a series of ranges
-  data.InitialMultiScan startMultiScan(
-    8:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    2:data.ScanBatch batch
-    3:list<data.TColumn> columns
-    4:list<data.IterInfo> ssiList
-    5:map<string, map<string, string>> ssio
-    6:list<binary> authorizations
-    7:bool waitForWrites
-    9:TSamplerConfiguration samplerConfig
-    10:i64 batchTimeOut
-    // name of the classloader context
-    11:string classLoaderContext
-    12:map<string, string> executionHints
-    13:i64 busyTimeout
-  ) throws (
-    1:client.ThriftSecurityException sec
-    2:TSampleNotPresentException tsnpe
-    3:ScanServerBusyException ssbe
-  )
-
-  data.MultiScanResult continueMultiScan(
-    2:trace.TInfo tinfo
-    1:data.ScanID scanID
-    3:i64 busyTimeout
-  ) throws (
-    1:NoSuchScanIDException nssi
-    2:TSampleNotPresentException tsnpe
-    3:ScanServerBusyException ssbe
-  )
-
-  void closeMultiScan(
-    2:trace.TInfo tinfo
-    1:data.ScanID scanID
-  ) throws (
-    1:NoSuchScanIDException nssi
-  )
-
-  list<ActiveScan> getActiveScans(
-    2:trace.TInfo tinfo
-    1:security.TCredentials credentials
-  ) throws (
-    1:client.ThriftSecurityException sec
-  )
-
-}
-
-service TabletClientService {
-
-  //the following calls support a batch update to multiple tablets on a tablet server
-  data.UpdateID startUpdate(
-    2:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    3:TDurability durability
-  ) throws (
-    1:client.ThriftSecurityException sec
-  )
-
-  oneway void applyUpdates(
-    1:trace.TInfo tinfo
-    2:data.UpdateID updateID
-    3:data.TKeyExtent keyExtent
-    4:list<data.TMutation> mutations
-  )
-
-  data.UpdateErrors closeUpdate(
-    2:trace.TInfo tinfo
-    1:data.UpdateID updateID
-  ) throws (
-    1:NoSuchScanIDException nssi
-  )
-
-  bool cancelUpdate(
-    1:trace.TInfo tinfo
-    2:data.UpdateID updateID
-  )
-
-  //the following call supports making a single update to a tablet
-  void update(
-    4:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    2:data.TKeyExtent keyExtent
-    3:data.TMutation mutation
-    5:TDurability durability
-  ) throws (
-    1:client.ThriftSecurityException sec
-    2:NotServingTabletException nste
-    3:ConstraintViolationException cve
-  )
-
-  data.TConditionalSession startConditionalUpdate(
-    1:trace.TInfo tinfo
-    2:security.TCredentials credentials
-    3:list<binary> authorizations
-    4:string tableID
-    5:TDurability durability
-    6:string classLoaderContext
-  ) throws (
-    1:client.ThriftSecurityException sec
-  )
-
-  list<data.TCMResult> conditionalUpdate(
-    1:trace.TInfo tinfo
-    2:data.UpdateID sessID
-    3:data.CMBatch mutations
-    4:list<string> symbols
-  ) throws (
-    1:NoSuchScanIDException nssi
-  )
-
-  void invalidateConditionalUpdate(
-    1:trace.TInfo tinfo
-    2:data.UpdateID sessID
-  )
-
-  oneway void closeConditionalUpdate(
-    1:trace.TInfo tinfo
-    2:data.UpdateID sessID
-  )
-
-  // on success, returns an empty list
-  list<data.TKeyExtent> bulkImport(
-    3:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    4:i64 tid
-    2:data.TabletFiles files
-    5:bool setTime
-  ) throws (
-    1:client.ThriftSecurityException sec
-  )
-
-  oneway void loadFiles(
-    1:trace.TInfo tinfo
-    2:security.TCredentials credentials
-    3:i64 tid
-    4:string dir
-    5:map<data.TKeyExtent, map<string, data.MapFileInfo>> files
-    6:bool setTime
-  )
-
-  void splitTablet(
-    4:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    2:data.TKeyExtent extent
-    3:binary splitPoint
-  ) throws (
-    1:client.ThriftSecurityException sec
-    2:NotServingTabletException nste
-  )
-
-  oneway void loadTablet(
-    5:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    4:string lock
-    2:data.TKeyExtent extent
-  )
-
-  oneway void unloadTablet(
-    5:trace.TInfo tinfo
-    1:security.TCredentials credentials
-    4:string lock
-    2:data.TKeyExtent extent
-    6:TUnloadTabletGoal goal
-    7:i64 requestTime
-  )
+service TabletServerClientService {
 
   oneway void flush(
-    4:trace.TInfo tinfo
+    4:client.TInfo tinfo
     1:security.TCredentials credentials
     3:string lock
     2:string tableId
@@ -421,22 +142,8 @@
     6:binary endRow
   )
 
-  oneway void flushTablet(
-    1:trace.TInfo tinfo
-    2:security.TCredentials credentials
-    3:string lock
-    4:data.TKeyExtent extent
-  )
-
-  oneway void chop(
-    1:trace.TInfo tinfo
-    2:security.TCredentials credentials
-    3:string lock
-    4:data.TKeyExtent extent
-  )
-
   oneway void compact(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string lock
     4:string tableId
@@ -444,15 +151,15 @@
     6:binary endRow
   )
 
-  master.TabletServerStatus getTabletServerStatus(
-    3:trace.TInfo tinfo
+  manager.TabletServerStatus getTabletServerStatus(
+    3:client.TInfo tinfo
     1:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
 
   list<TabletStats> getTabletStats(
-    3:trace.TInfo tinfo
+    3:client.TInfo tinfo
     1:security.TCredentials credentials
     2:string tableId
   ) throws (
@@ -460,14 +167,14 @@
   )
 
   TabletStats getHistoricalStats(
-    2:trace.TInfo tinfo
+    2:client.TInfo tinfo
     1:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
 
   void halt(
-    3:trace.TInfo tinfo
+    3:client.TInfo tinfo
     1:security.TCredentials credentials
     2:string lock
   ) throws (
@@ -475,31 +182,31 @@
   )
 
   oneway void fastHalt(
-    3:trace.TInfo tinfo
+    3:client.TInfo tinfo
     1:security.TCredentials credentials
     2:string lock
   )
 
   list<ActiveCompaction> getActiveCompactions(
-    2:trace.TInfo tinfo
+    2:client.TInfo tinfo
     1:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
 
   oneway void removeLogs(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:list<string> filenames
   )
 
   list<string> getActiveLogs(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   )
 
   data.TSummaries startGetSummaries(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:data.TSummaryRequest request
   ) throws (
@@ -508,7 +215,7 @@
   )
 
   data.TSummaries startGetSummariesForPartition(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:data.TSummaryRequest request
     4:i32 modulus
@@ -518,7 +225,7 @@
   )
 
   data.TSummaries startGetSummariesFromFiles(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:data.TSummaryRequest request
     4:map<string, list<data.TRowRange>> files
@@ -527,21 +234,21 @@
   )
 
   data.TSummaries contiuneGetSummaries(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:i64 sessionId
   ) throws (
     1:NoSuchScanIDException nssi
   )
   
   list<TCompactionQueueSummary> getCompactionQueueInfo(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
   ) throws (
     1:client.ThriftSecurityException sec
   )
   
   TExternalCompactionJob reserveCompactionJob(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string queueName
     4:i64 priority
@@ -552,7 +259,7 @@
   )
   
   oneway void compactionJobFinished(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string externalCompactionId
     4:data.TKeyExtent extent
@@ -561,7 +268,7 @@
   )
 
   oneway void compactionJobFailed(
-    1:trace.TInfo tinfo
+    1:client.TInfo tinfo
     2:security.TCredentials credentials
     3:string externalCompactionId
     4:data.TKeyExtent extent
diff --git a/core/src/test/java/org/apache/accumulo/core/classloader/ContextClassLoaderFactoryTest.java b/core/src/test/java/org/apache/accumulo/core/classloader/ContextClassLoaderFactoryTest.java
index 83b3821..ffc9a8a 100644
--- a/core/src/test/java/org/apache/accumulo/core/classloader/ContextClassLoaderFactoryTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/classloader/ContextClassLoaderFactoryTest.java
@@ -68,7 +68,7 @@
 
     ConfigurationCopy cc = new ConfigurationCopy();
     cc.set(Property.GENERAL_CONTEXT_CLASSLOADER_FACTORY.getKey(),
-        URLClassLoaderFactory.class.getName());
+        URLContextClassLoaderFactory.class.getName());
     ClassLoaderUtil.resetContextFactoryForTests();
     ClassLoaderUtil.initContextFactory(cc);
 
diff --git a/core/src/test/java/org/apache/accumulo/core/classloader/URLClassLoaderFactory.java b/core/src/test/java/org/apache/accumulo/core/classloader/URLClassLoaderFactory.java
deleted file mode 100644
index c97a7d2..0000000
--- a/core/src/test/java/org/apache/accumulo/core/classloader/URLClassLoaderFactory.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.classloader;
-
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.stream.Stream;
-
-import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
-
-// test implementation
-public class URLClassLoaderFactory implements ContextClassLoaderFactory {
-
-  private static final String COMMA = ",";
-
-  @Override
-  public ClassLoader getClassLoader(String contextName) {
-    // The context name is the classpath.
-    URL[] urls = Stream.of(contextName.split(COMMA)).map(p -> {
-      try {
-        return new URL(p);
-      } catch (MalformedURLException e) {
-        throw new IllegalArgumentException("Error creating URL from classpath segment: " + p, e);
-      }
-    }).toArray(URL[]::new);
-    return URLClassLoader.newInstance(urls);
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java b/core/src/test/java/org/apache/accumulo/core/cli/ConfigOptsTest.java
similarity index 73%
rename from server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
rename to core/src/test/java/org/apache/accumulo/core/cli/ConfigOptsTest.java
index 1cde30e..b5a38de 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/cli/ConfigOptsTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server;
+package org.apache.accumulo.core.cli;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
@@ -26,32 +26,34 @@
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-public class ServerOptsTest {
-  private ServerOpts opts;
+public class ConfigOptsTest {
+  private ConfigOpts opts;
 
   @BeforeEach
   public void setUp() {
-    opts = new ServerOpts();
+    opts = new ConfigOpts();
   }
 
   @Test
   public void testGetAddress() {
-    opts.parseArgs(ServerOptsTest.class.getName(), new String[] {"-a", "1.2.3.4"});
-    assertEquals("1.2.3.4", opts.getAddress());
+    opts.parseArgs(ConfigOptsTest.class.getName(),
+        new String[] {"-o", Property.GENERAL_PROCESS_BIND_ADDRESS.getKey() + "=1.2.3.4"});
+    assertEquals("1.2.3.4", opts.getSiteConfiguration().get(Property.GENERAL_PROCESS_BIND_ADDRESS));
   }
 
   @Test
   public void testGetAddress_NOne() {
-    opts.parseArgs(ServerOptsTest.class.getName(), new String[] {});
-    assertEquals("0.0.0.0", opts.getAddress());
+    opts.parseArgs(ConfigOptsTest.class.getName(), new String[] {});
+    assertEquals("0.0.0.0", opts.getSiteConfiguration().get(Property.GENERAL_PROCESS_BIND_ADDRESS));
   }
 
   @Test
   public void testOverrideConfig() {
     AccumuloConfiguration defaults = DefaultConfiguration.getInstance();
     assertEquals("localhost:2181", defaults.get(Property.INSTANCE_ZK_HOST));
-    opts.parseArgs(ServerOptsTest.class.getName(),
+    opts.parseArgs(ConfigOptsTest.class.getName(),
         new String[] {"-o", "instance.zookeeper.host=test:123"});
     assertEquals("test:123", opts.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST));
   }
+
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/cli/PasswordConverterTest.java b/core/src/test/java/org/apache/accumulo/core/cli/PasswordConverterTest.java
index 6470626..0763973 100644
--- a/core/src/test/java/org/apache/accumulo/core/cli/PasswordConverterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/cli/PasswordConverterTest.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.cli;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
@@ -28,7 +29,6 @@
 import java.io.OutputStreamWriter;
 import java.io.PipedInputStream;
 import java.io.PipedOutputStream;
-import java.security.SecureRandom;
 import java.util.Scanner;
 
 import org.junit.jupiter.api.AfterEach;
@@ -42,8 +42,6 @@
 
 public class PasswordConverterTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
   private class Password {
     @Parameter(names = "--password", converter = ClientOpts.PasswordConverter.class)
     String password;
@@ -79,7 +77,7 @@
 
   @Test
   public void testPass() {
-    String expected = String.valueOf(random.nextDouble());
+    String expected = String.valueOf(RANDOM.get().nextDouble());
     argv[1] = "pass:" + expected;
     new JCommander(password).parse(argv);
     assertEquals(expected, password.password);
diff --git a/core/src/test/java/org/apache/accumulo/core/cli/TestHelp.java b/core/src/test/java/org/apache/accumulo/core/cli/TestHelp.java
index 7d9c2d4..974d7f3 100644
--- a/core/src/test/java/org/apache/accumulo/core/cli/TestHelp.java
+++ b/core/src/test/java/org/apache/accumulo/core/cli/TestHelp.java
@@ -31,7 +31,7 @@
 
     @Override
     public void exit(int status) {
-      throw new RuntimeException(Integer.toString(status));
+      throw new IllegalStateException(Integer.toString(status));
     }
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.java
deleted file mode 100644
index 943d9aa..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/ClientConfigurationTest.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.client;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("deprecation")
-public class ClientConfigurationTest {
-
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty INSTANCE_NAME =
-      org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_NAME;
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty INSTANCE_ZK_HOST =
-      org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST;
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty INSTANCE_ZK_TIMEOUT =
-      org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT;
-  private static org.apache.accumulo.core.client.ClientConfiguration.ClientProperty RPC_SSL_TRUSTSTORE_TYPE =
-      ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_TYPE;
-
-  @Test
-  public void testOverrides() {
-    ClientConfiguration clientConfig = createConfig();
-    assertExpectedConfig(clientConfig);
-  }
-
-  @Test
-  public void testSerialization() {
-    ClientConfiguration clientConfig = createConfig();
-    // sanity check that we're starting with what we're expecting
-    assertExpectedConfig(clientConfig);
-
-    String serialized = clientConfig.serialize();
-    ClientConfiguration deserializedClientConfig = ClientConfiguration.deserialize(serialized);
-    assertExpectedConfig(deserializedClientConfig);
-  }
-
-  private void assertExpectedConfig(ClientConfiguration clientConfig) {
-    assertEquals("firstZkHosts", clientConfig.get(INSTANCE_ZK_HOST));
-    assertEquals("secondInstanceName", clientConfig.get(INSTANCE_NAME));
-    assertEquals("123s", clientConfig.get(INSTANCE_ZK_TIMEOUT));
-    assertEquals(RPC_SSL_TRUSTSTORE_TYPE.getDefaultValue(),
-        clientConfig.get(RPC_SSL_TRUSTSTORE_TYPE));
-  }
-
-  private ClientConfiguration createConfig() {
-    return ClientConfiguration.create().with(INSTANCE_ZK_HOST, "firstZkHosts")
-        .with(INSTANCE_NAME, "secondInstanceName").with(INSTANCE_ZK_TIMEOUT, "123s");
-  }
-
-  @Test
-  public void testConfPath() throws IOException {
-    File target = new File(System.getProperty("user.dir"), "target");
-    assertTrue(target.exists(), "'target' build directory does not exist");
-    File testDir = new File(target, getClass().getName());
-    if (!testDir.exists()) {
-      assertTrue(testDir.mkdirs(), "Failed to create test dir " + testDir);
-    }
-
-    File clientConf = new File(testDir, "client.conf");
-    if (!clientConf.exists()) {
-      assertTrue(clientConf.createNewFile(), "Failed to create file " + clientConf);
-    }
-
-    // A directory should return the path with client.conf appended.
-    assertEquals(clientConf.toString(), ClientConfiguration.getClientConfPath(testDir.toString()));
-    // A normal file should return itself
-    assertEquals(clientConf.toString(),
-        ClientConfiguration.getClientConfPath(clientConf.toString()));
-
-    // Something that doesn't exist should return itself (specifically, it shouldn't error)
-    final File missing = new File("foobarbaz12332112");
-    assertEquals(missing.toString(), ClientConfiguration.getClientConfPath(missing.toString()));
-
-    assertNull(ClientConfiguration.getClientConfPath(null));
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/TestThrift1474.java b/core/src/test/java/org/apache/accumulo/core/client/TestThrift1474.java
index 42929cc..16ad82d 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/TestThrift1474.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/TestThrift1474.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.client;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -74,7 +73,7 @@
     Thread thread = new Thread(server::serve);
     thread.start();
     while (!server.isServing()) {
-      sleepUninterruptibly(10, MILLISECONDS);
+      Thread.sleep(10);
     }
 
     TTransport transport = new TSocket("localhost", port);
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
deleted file mode 100644
index 48047c7..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
+++ /dev/null
@@ -1,197 +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 java.nio.charset.StandardCharsets.UTF_8;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-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.easymock.EasyMock;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class ZooKeeperInstanceTest {
-  private static final InstanceId IID = InstanceId.of(UUID.randomUUID());
-  private static final String IID_STRING = IID.canonical();
-  private ZooCacheFactory zcf;
-  private ZooCache zc;
-  private ZooKeeperInstance zki;
-
-  private static final ClientConfiguration.ClientProperty INSTANCE_ID =
-      ClientConfiguration.ClientProperty.INSTANCE_ID;
-  private static final ClientConfiguration.ClientProperty INSTANCE_NAME =
-      ClientConfiguration.ClientProperty.INSTANCE_NAME;
-  private static final ClientConfiguration.ClientProperty INSTANCE_ZK_HOST =
-      ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST;
-  private static final ClientConfiguration.ClientProperty INSTANCE_ZK_TIMEOUT =
-      ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT;
-
-  private void mockIdConstruction(ClientConfiguration config) {
-    expect(config.get(INSTANCE_ID)).andReturn(IID_STRING);
-    expect(config.get(INSTANCE_NAME)).andReturn(null);
-    expect(config.get(INSTANCE_ZK_HOST)).andReturn("zk1");
-    expect(config.get(INSTANCE_ZK_TIMEOUT)).andReturn("30");
-  }
-
-  private void mockNameConstruction(ClientConfiguration config) {
-    expect(config.get(INSTANCE_ID)).andReturn(null);
-    expect(config.get(INSTANCE_NAME)).andReturn("instance");
-    expect(config.get(INSTANCE_ZK_HOST)).andReturn("zk1");
-    expect(config.get(INSTANCE_ZK_TIMEOUT)).andReturn("30");
-  }
-
-  @BeforeEach
-  public void setUp() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    mockNameConstruction(config);
-    replay(config);
-    zcf = createMock(ZooCacheFactory.class);
-    zc = createMock(ZooCache.class);
-    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc).anyTimes();
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance"))
-        .andReturn(IID_STRING.getBytes(UTF_8));
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
-    replay(zc, zcf);
-    zki = new ZooKeeperInstance(config, zcf);
-    EasyMock.resetToDefault(zc);
-  }
-
-  @Test
-  public void testInvalidConstruction() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    expect(config.get(INSTANCE_ID)).andReturn(IID_STRING);
-    mockNameConstruction(config);
-    replay(config);
-    assertThrows(IllegalArgumentException.class, () -> new ZooKeeperInstance(config));
-  }
-
-  @Test
-  public void testInvalidConstruction2() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    expect(config.get(INSTANCE_ID)).andReturn(null);
-    expect(config.get(INSTANCE_NAME)).andReturn(null);
-    expect(config.get(INSTANCE_ZK_HOST)).andReturn("zk1");
-    expect(config.get(INSTANCE_ZK_TIMEOUT)).andReturn("30");
-    replay(config);
-    assertThrows(IllegalArgumentException.class, () -> new ZooKeeperInstance(config));
-  }
-
-  @Test
-  public void testSimpleGetters() {
-    assertEquals("instance", zki.getInstanceName());
-    assertEquals("zk1", zki.getZooKeepers());
-    assertEquals(30000, zki.getZooKeepersSessionTimeOut());
-  }
-
-  @Test
-  public void testGetInstanceID_FromCache() {
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance"))
-        .andReturn(IID_STRING.getBytes(UTF_8));
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
-    replay(zc);
-    assertEquals(IID_STRING, zki.getInstanceID());
-  }
-
-  @Test
-  public void testGetInstanceID_Direct() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    mockIdConstruction(config);
-    replay(config);
-    zki = new ZooKeeperInstance(config, zcf);
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
-    replay(zc);
-    assertEquals(IID_STRING, zki.getInstanceID());
-  }
-
-  @Test
-  public void testGetInstanceID_NoMapping() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(null);
-    replay(zc);
-    EasyMock.reset(config, zcf);
-    assertThrows(RuntimeException.class, () -> new ZooKeeperInstance(config, zcf));
-  }
-
-  @Test
-  public void testGetInstanceID_IDMissingForName() {
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance"))
-        .andReturn(IID_STRING.getBytes(UTF_8));
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
-    replay(zc);
-    assertThrows(RuntimeException.class, () -> zki.getInstanceID());
-  }
-
-  @Test
-  public void testGetInstanceID_IDMissingForID() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    mockIdConstruction(config);
-    replay(config);
-    zki = new ZooKeeperInstance(config, zcf);
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
-    replay(zc);
-    assertThrows(RuntimeException.class, () -> zki.getInstanceID());
-  }
-
-  @Test
-  public void testGetInstanceName() {
-    ClientConfiguration config = createMock(ClientConfiguration.class);
-    mockIdConstruction(config);
-    replay(config);
-    zki = new ZooKeeperInstance(config, zcf);
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
-    List<String> children = new java.util.ArrayList<>();
-    children.add("child1");
-    children.add("child2");
-    expect(zc.getChildren(Constants.ZROOT + Constants.ZINSTANCES)).andReturn(children);
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1"))
-        .andReturn(UUID.randomUUID().toString().getBytes(UTF_8));
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2"))
-        .andReturn(IID_STRING.getBytes(UTF_8));
-    replay(zc);
-    assertEquals("child2", zki.getInstanceName());
-  }
-
-  @Test
-  public void testAllZooKeepersAreUsed() {
-    final String zookeepers = "zk1,zk2,zk3", instanceName = "accumulo";
-    ZooCacheFactory factory = createMock(ZooCacheFactory.class);
-    EasyMock.reset(zc);
-    expect(factory.getZooCache(zookeepers, 30000)).andReturn(zc).anyTimes();
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName))
-        .andReturn(IID_STRING.getBytes(UTF_8));
-    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
-    replay(zc, factory);
-    ClientConfiguration cfg =
-        ClientConfiguration.loadDefault().withInstance(instanceName).withZkHosts(zookeepers);
-    ZooKeeperInstance zki = new ZooKeeperInstance(cfg, factory);
-    assertEquals(zookeepers, zki.getZooKeepers());
-    assertEquals(instanceName, zki.getInstanceName());
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
deleted file mode 100644
index 8982380..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
+++ /dev/null
@@ -1,95 +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.junit.jupiter.api.Assertions.assertEquals;
-
-import org.apache.accumulo.core.client.sample.RowSampler;
-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.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.mapred.JobConf;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormatTest {
-
-  @Test
-  public void validateConfiguration() {
-
-    int a = 7;
-    long b = 300L;
-    long c = 50L;
-    long d = 10L;
-    String e = "snappy";
-    SamplerConfiguration samplerConfig = new SamplerConfiguration(RowSampler.class.getName());
-    samplerConfig.addOption("hasher", "murmur3_32");
-    samplerConfig.addOption("modulus", "109");
-
-    JobConf job = new JobConf();
-    AccumuloFileOutputFormat.setReplication(job, a);
-    AccumuloFileOutputFormat.setFileBlockSize(job, b);
-    AccumuloFileOutputFormat.setDataBlockSize(job, c);
-    AccumuloFileOutputFormat.setIndexBlockSize(job, d);
-    AccumuloFileOutputFormat.setCompressionType(job, e);
-    AccumuloFileOutputFormat.setSampler(job, samplerConfig);
-
-    AccumuloConfiguration acuconf =
-        org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
-            .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
-
-    assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
-    assertEquals(300L, acuconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE));
-    assertEquals(50L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
-    assertEquals(10L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
-    assertEquals("snappy", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
-    assertEquals(new SamplerConfigurationImpl(samplerConfig),
-        SamplerConfigurationImpl.newSamplerConfig(acuconf));
-
-    a = 17;
-    b = 1300L;
-    c = 150L;
-    d = 110L;
-    e = "lzo";
-    samplerConfig = new SamplerConfiguration(RowSampler.class.getName());
-    samplerConfig.addOption("hasher", "md5");
-    samplerConfig.addOption("modulus", "100003");
-
-    job = new JobConf();
-    AccumuloFileOutputFormat.setReplication(job, a);
-    AccumuloFileOutputFormat.setFileBlockSize(job, b);
-    AccumuloFileOutputFormat.setDataBlockSize(job, c);
-    AccumuloFileOutputFormat.setIndexBlockSize(job, d);
-    AccumuloFileOutputFormat.setCompressionType(job, e);
-    AccumuloFileOutputFormat.setSampler(job, samplerConfig);
-
-    acuconf = org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
-        .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
-
-    assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
-    assertEquals(1300L, acuconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE));
-    assertEquals(150L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
-    assertEquals(110L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
-    assertEquals("lzo", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
-    assertEquals(new SamplerConfigurationImpl(samplerConfig),
-        SamplerConfigurationImpl.newSamplerConfig(acuconf));
-
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
deleted file mode 100644
index 91a507d..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ /dev/null
@@ -1,182 +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.junit.jupiter.api.Assertions.assertEquals;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Base64;
-import java.util.List;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.iterators.user.RegExFilter;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.hadoop.mapred.JobConf;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormatTest {
-
-  private JobConf job;
-
-  @BeforeEach
-  public void createJob() {
-    job = new JobConf();
-  }
-
-  /**
-   * Check that the iterator configuration is getting stored in the Job conf correctly.
-   */
-  @Test
-  public void testSetIterator() throws IOException {
-    IteratorSetting is =
-        new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator");
-    AccumuloInputFormat.addIterator(job, is);
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    is.write(new DataOutputStream(baos));
-    String iterators = job.get("AccumuloInputFormat.ScanOpts.Iterators");
-    assertEquals(Base64.getEncoder().encodeToString(baos.toByteArray()), iterators);
-  }
-
-  @Test
-  public void testAddIterator() {
-    AccumuloInputFormat.addIterator(job,
-        new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
-    AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions",
-        "org.apache.accumulo.core.iterators.VersioningIterator"));
-    IteratorSetting iter =
-        new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator");
-    iter.addOption("v1", "1");
-    iter.addOption("junk", "\0omg:!\\xyzzy");
-    AccumuloInputFormat.addIterator(job, iter);
-
-    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-
-    // Check the list size
-    assertEquals(3, list.size());
-
-    // Walk the list and make sure our settings are correct
-    IteratorSetting setting = list.get(0);
-    assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator",
-        setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getName());
-    assertEquals(0, setting.getOptions().size());
-
-    setting = list.get(1);
-    assertEquals(2, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator",
-        setting.getIteratorClass());
-    assertEquals("Versions", setting.getName());
-    assertEquals(0, setting.getOptions().size());
-
-    setting = list.get(2);
-    assertEquals(3, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getName());
-    assertEquals(2, setting.getOptions().size());
-    assertEquals("1", setting.getOptions().get("v1"));
-    assertEquals("\0omg:!\\xyzzy", setting.getOptions().get("junk"));
-  }
-
-  /**
-   * Test adding iterator options where the keys and values contain both the FIELD_SEPARATOR
-   * character (':') and ITERATOR_SEPARATOR (',') characters. There should be no exceptions thrown
-   * when trying to parse these types of option entries.
-   *
-   * This test makes sure that the expected raw values, as appears in the Job, are equal to what's
-   * expected.
-   */
-  @Test
-  public void testIteratorOptionEncoding() {
-    String key = "colon:delimited:key";
-    String value = "comma,delimited,value";
-    IteratorSetting someSetting = new IteratorSetting(1, "iterator", "Iterator.class");
-    someSetting.addOption(key, value);
-    AccumuloInputFormat.addIterator(job, someSetting);
-
-    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-    assertEquals(1, list.size());
-    assertEquals(1, list.get(0).getOptions().size());
-    assertEquals(list.get(0).getOptions().get(key), value);
-
-    someSetting.addOption(key + "2", value);
-    someSetting.setPriority(2);
-    someSetting.setName("it2");
-    AccumuloInputFormat.addIterator(job, someSetting);
-    list = AccumuloInputFormat.getIterators(job);
-    assertEquals(2, list.size());
-    assertEquals(1, list.get(0).getOptions().size());
-    assertEquals(list.get(0).getOptions().get(key), value);
-    assertEquals(2, list.get(1).getOptions().size());
-    assertEquals(list.get(1).getOptions().get(key), value);
-    assertEquals(list.get(1).getOptions().get(key + "2"), value);
-  }
-
-  /**
-   * Test getting iterator settings for multiple iterators set
-   */
-  @Test
-  public void testGetIteratorSettings() {
-    AccumuloInputFormat.addIterator(job,
-        new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator"));
-    AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions",
-        "org.apache.accumulo.core.iterators.VersioningIterator"));
-    AccumuloInputFormat.addIterator(job,
-        new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator"));
-
-    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-
-    // Check the list size
-    assertEquals(3, list.size());
-
-    // Walk the list and make sure our settings are correct
-    IteratorSetting setting = list.get(0);
-    assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getName());
-
-    setting = list.get(1);
-    assertEquals(2, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator",
-        setting.getIteratorClass());
-    assertEquals("Versions", setting.getName());
-
-    setting = list.get(2);
-    assertEquals(3, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getName());
-
-  }
-
-  @Test
-  public void testSetRegex() {
-    String regex = ">\"*%<>\'\\";
-
-    IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
-    RegExFilter.setRegexs(is, regex, null, null, null, false);
-    AccumuloInputFormat.addIterator(job, is);
-
-    assertEquals(regex, AccumuloInputFormat.getIterators(job).get(0).getName());
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
deleted file mode 100644
index 1b40d6c..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
+++ /dev/null
@@ -1,71 +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.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.WithTestNames;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormatTest extends WithTestNames {
-
-  /**
-   * Verify {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects get correctly
-   * serialized in the JobContext.
-   */
-  @Test
-  public void testTableQueryConfigSerialization() {
-    String table1Name = testName() + "1";
-    String table2Name = testName() + "2";
-    JobConf job = new JobConf();
-
-    org.apache.accumulo.core.client.mapreduce.InputTableConfig table1 =
-        new org.apache.accumulo.core.client.mapreduce.InputTableConfig()
-            .setRanges(Collections.singletonList(new Range("a", "b")))
-            .fetchColumns(Collections.singleton(new Pair<>(new Text("CF1"), new Text("CQ1"))))
-            .setIterators(
-                Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
-    org.apache.accumulo.core.client.mapreduce.InputTableConfig table2 =
-        new org.apache.accumulo.core.client.mapreduce.InputTableConfig()
-            .setRanges(Collections.singletonList(new Range("a", "b")))
-            .fetchColumns(Collections.singleton(new Pair<>(new Text("CF1"), new Text("CQ1"))))
-            .setIterators(
-                Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
-    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configMap =
-        new HashMap<>();
-    configMap.put(table1Name, table1);
-    configMap.put(table2Name, table2);
-    AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
-
-    assertEquals(table1, AccumuloMultiTableInputFormat.getInputTableConfig(job, table1Name));
-    assertEquals(table2, AccumuloMultiTableInputFormat.getInputTableConfig(job, table2Name));
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
deleted file mode 100644
index 35fc29a..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
+++ /dev/null
@@ -1,75 +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 java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.mapred.JobConf;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormatTest {
-
-  @Test
-  public void testBWSettings() throws IOException {
-    JobConf job = new JobConf();
-
-    // make sure we aren't testing defaults
-    final BatchWriterConfig bwDefaults = new BatchWriterConfig();
-    assertNotEquals(7654321L, bwDefaults.getMaxLatency(MILLISECONDS));
-    assertNotEquals(9898989L, bwDefaults.getTimeout(MILLISECONDS));
-    assertNotEquals(42, bwDefaults.getMaxWriteThreads());
-    assertNotEquals(1123581321L, bwDefaults.getMaxMemory());
-
-    final BatchWriterConfig bwConfig = new BatchWriterConfig();
-    bwConfig.setMaxLatency(7654321L, MILLISECONDS);
-    bwConfig.setTimeout(9898989L, MILLISECONDS);
-    bwConfig.setMaxWriteThreads(42);
-    bwConfig.setMaxMemory(1123581321L);
-    AccumuloOutputFormat.setBatchWriterOptions(job, bwConfig);
-
-    AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
-      @Override
-      public void checkOutputSpecs(FileSystem ignored, JobConf job) {
-        BatchWriterConfig bwOpts = getBatchWriterOptions(job);
-
-        // passive check
-        assertEquals(bwConfig.getMaxLatency(MILLISECONDS), bwOpts.getMaxLatency(MILLISECONDS));
-        assertEquals(bwConfig.getTimeout(MILLISECONDS), bwOpts.getTimeout(MILLISECONDS));
-        assertEquals(bwConfig.getMaxWriteThreads(), bwOpts.getMaxWriteThreads());
-        assertEquals(bwConfig.getMaxMemory(), bwOpts.getMaxMemory());
-
-        // explicit check
-        assertEquals(7654321L, bwOpts.getMaxLatency(MILLISECONDS));
-        assertEquals(9898989L, bwOpts.getTimeout(MILLISECONDS));
-        assertEquals(42, bwOpts.getMaxWriteThreads());
-        assertEquals(1123581321L, bwOpts.getMaxMemory());
-
-      }
-    };
-    myAOF.checkOutputSpecs(null, job);
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
deleted file mode 100644
index 97663c7..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
+++ /dev/null
@@ -1,115 +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.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-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.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.iterators.user.SummingCombiner;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class RangeInputSplitTest {
-
-  @Test
-  public void testSimpleWritable() throws IOException {
-    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new Key("a"), new Key("b")),
-        new String[] {"localhost"});
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-
-    RangeInputSplit newSplit = new RangeInputSplit();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-
-    assertEquals(split.getRange(), newSplit.getRange());
-    assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
-  }
-
-  @Test
-  public void testAllFieldsWritable() throws IOException {
-    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new Key("a"), new Key("b")),
-        new String[] {"localhost"});
-
-    Set<Pair<Text,Text>> fetchedColumns = new HashSet<>();
-
-    fetchedColumns.add(new Pair<>(new Text("colf1"), new Text("colq1")));
-    fetchedColumns.add(new Pair<>(new Text("colf2"), new Text("colq2")));
-
-    // Fake some iterators
-    ArrayList<IteratorSetting> iterators = new ArrayList<>();
-    IteratorSetting setting = new IteratorSetting(50, SummingCombiner.class);
-    setting.addOption("foo", "bar");
-    iterators.add(setting);
-
-    setting = new IteratorSetting(100, WholeRowIterator.class);
-    setting.addOption("bar", "foo");
-    iterators.add(setting);
-
-    split.setOffline(true);
-    split.setIsolatedScan(true);
-    split.setUsesLocalIterators(true);
-    split.setFetchedColumns(fetchedColumns);
-    split.setIterators(iterators);
-    split.setLogLevel(Level.WARN);
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-
-    RangeInputSplit newSplit = new RangeInputSplit();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-
-    assertEquals(split.getRange(), newSplit.getRange());
-    assertArrayEquals(split.getLocations(), newSplit.getLocations());
-
-    assertEquals(split.isOffline(), newSplit.isOffline());
-    assertEquals(split.isIsolatedScan(), newSplit.isOffline());
-    assertEquals(split.usesLocalIterators(), newSplit.usesLocalIterators());
-    assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
-    assertEquals(split.getIterators(), newSplit.getIterators());
-    assertEquals(split.getLogLevel(), newSplit.getLogLevel());
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
deleted file mode 100644
index 5b0319d..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
+++ /dev/null
@@ -1,97 +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.junit.jupiter.api.Assertions.assertEquals;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.sample.RowSampler;
-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.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormatTest {
-
-  @Test
-  public void validateConfiguration() throws IOException {
-
-    int a = 7;
-    long b = 300L;
-    long c = 50L;
-    long d = 10L;
-    String e = "snappy";
-    SamplerConfiguration samplerConfig = new SamplerConfiguration(RowSampler.class.getName());
-    samplerConfig.addOption("hasher", "murmur3_32");
-    samplerConfig.addOption("modulus", "109");
-
-    Job job1 = Job.getInstance();
-    AccumuloFileOutputFormat.setReplication(job1, a);
-    AccumuloFileOutputFormat.setFileBlockSize(job1, b);
-    AccumuloFileOutputFormat.setDataBlockSize(job1, c);
-    AccumuloFileOutputFormat.setIndexBlockSize(job1, d);
-    AccumuloFileOutputFormat.setCompressionType(job1, e);
-    AccumuloFileOutputFormat.setSampler(job1, samplerConfig);
-
-    AccumuloConfiguration acuconf =
-        org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
-            .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job1.getConfiguration());
-
-    assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
-    assertEquals(300L, acuconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE));
-    assertEquals(50L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
-    assertEquals(10L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
-    assertEquals("snappy", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
-    assertEquals(new SamplerConfigurationImpl(samplerConfig),
-        SamplerConfigurationImpl.newSamplerConfig(acuconf));
-
-    a = 17;
-    b = 1300L;
-    c = 150L;
-    d = 110L;
-    e = "lzo";
-    samplerConfig = new SamplerConfiguration(RowSampler.class.getName());
-    samplerConfig.addOption("hasher", "md5");
-    samplerConfig.addOption("modulus", "100003");
-
-    Job job2 = Job.getInstance();
-    AccumuloFileOutputFormat.setReplication(job2, a);
-    AccumuloFileOutputFormat.setFileBlockSize(job2, b);
-    AccumuloFileOutputFormat.setDataBlockSize(job2, c);
-    AccumuloFileOutputFormat.setIndexBlockSize(job2, d);
-    AccumuloFileOutputFormat.setCompressionType(job2, e);
-    AccumuloFileOutputFormat.setSampler(job2, samplerConfig);
-
-    acuconf = org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
-        .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job2.getConfiguration());
-
-    assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
-    assertEquals(1300L, acuconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE));
-    assertEquals(150L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE));
-    assertEquals(110L, acuconf.getAsBytes(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX));
-    assertEquals("lzo", acuconf.get(Property.TABLE_FILE_COMPRESSION_TYPE));
-    assertEquals(new SamplerConfigurationImpl(samplerConfig),
-        SamplerConfigurationImpl.newSamplerConfig(acuconf));
-
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
deleted file mode 100644
index 801f6a6..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ /dev/null
@@ -1,202 +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.junit.jupiter.api.Assertions.assertEquals;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Base64;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.iterators.user.RegExFilter;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormatTest {
-
-  /**
-   * Check that the iterator configuration is getting stored in the Job conf correctly.
-   */
-  @Test
-  public void testSetIterator() throws IOException {
-    Job job = Job.getInstance();
-
-    IteratorSetting is =
-        new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator");
-    AccumuloInputFormat.addIterator(job, is);
-    Configuration conf = job.getConfiguration();
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    is.write(new DataOutputStream(baos));
-    String iterators = conf.get("AccumuloInputFormat.ScanOpts.Iterators");
-    assertEquals(Base64.getEncoder().encodeToString(baos.toByteArray()), iterators);
-  }
-
-  @Test
-  public void testAddIterator() throws IOException {
-    Job job = Job.getInstance();
-
-    AccumuloInputFormat.addIterator(job,
-        new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
-    AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions",
-        "org.apache.accumulo.core.iterators.VersioningIterator"));
-    IteratorSetting iter =
-        new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator");
-    iter.addOption("v1", "1");
-    iter.addOption("junk", "\0omg:!\\xyzzy");
-    AccumuloInputFormat.addIterator(job, iter);
-
-    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-
-    // Check the list size
-    assertEquals(3, list.size());
-
-    // Walk the list and make sure our settings are correct
-    IteratorSetting setting = list.get(0);
-    assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator",
-        setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getName());
-    assertEquals(0, setting.getOptions().size());
-
-    setting = list.get(1);
-    assertEquals(2, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator",
-        setting.getIteratorClass());
-    assertEquals("Versions", setting.getName());
-    assertEquals(0, setting.getOptions().size());
-
-    setting = list.get(2);
-    assertEquals(3, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getName());
-    assertEquals(2, setting.getOptions().size());
-    assertEquals("1", setting.getOptions().get("v1"));
-    assertEquals("\0omg:!\\xyzzy", setting.getOptions().get("junk"));
-  }
-
-  /**
-   * Test adding iterator options where the keys and values contain both the FIELD_SEPARATOR
-   * character (':') and ITERATOR_SEPARATOR (',') characters. There should be no exceptions thrown
-   * when trying to parse these types of option entries.
-   *
-   * This test makes sure that the expected raw values, as appears in the Job, are equal to what's
-   * expected.
-   */
-  @Test
-  public void testIteratorOptionEncoding() throws Throwable {
-    String key = "colon:delimited:key";
-    String value = "comma,delimited,value";
-    IteratorSetting someSetting = new IteratorSetting(1, "iterator", "Iterator.class");
-    someSetting.addOption(key, value);
-    Job job = Job.getInstance();
-    AccumuloInputFormat.addIterator(job, someSetting);
-
-    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-    assertEquals(1, list.size());
-    assertEquals(1, list.get(0).getOptions().size());
-    assertEquals(list.get(0).getOptions().get(key), value);
-
-    someSetting.addOption(key + "2", value);
-    someSetting.setPriority(2);
-    someSetting.setName("it2");
-    AccumuloInputFormat.addIterator(job, someSetting);
-    list = AccumuloInputFormat.getIterators(job);
-    assertEquals(2, list.size());
-    assertEquals(1, list.get(0).getOptions().size());
-    assertEquals(list.get(0).getOptions().get(key), value);
-    assertEquals(2, list.get(1).getOptions().size());
-    assertEquals(list.get(1).getOptions().get(key), value);
-    assertEquals(list.get(1).getOptions().get(key + "2"), value);
-  }
-
-  /**
-   * Test getting iterator settings for multiple iterators set
-   */
-  @Test
-  public void testGetIteratorSettings() throws IOException {
-    Job job = Job.getInstance();
-
-    AccumuloInputFormat.addIterator(job,
-        new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator"));
-    AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions",
-        "org.apache.accumulo.core.iterators.VersioningIterator"));
-    AccumuloInputFormat.addIterator(job,
-        new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator"));
-
-    List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-
-    // Check the list size
-    assertEquals(3, list.size());
-
-    // Walk the list and make sure our settings are correct
-    IteratorSetting setting = list.get(0);
-    assertEquals(1, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
-    assertEquals("WholeRow", setting.getName());
-
-    setting = list.get(1);
-    assertEquals(2, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.VersioningIterator",
-        setting.getIteratorClass());
-    assertEquals("Versions", setting.getName());
-
-    setting = list.get(2);
-    assertEquals(3, setting.getPriority());
-    assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
-    assertEquals("Count", setting.getName());
-
-  }
-
-  @Test
-  public void testSetRegex() throws IOException {
-    Job job = Job.getInstance();
-
-    String regex = ">\"*%<>\'\\";
-
-    IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
-    RegExFilter.setRegexs(is, regex, null, null, null, false);
-    AccumuloInputFormat.addIterator(job, is);
-
-    assertEquals(regex, AccumuloInputFormat.getIterators(job).get(0).getName());
-  }
-
-  @Test
-  public void testEmptyColumnFamily() throws IOException {
-    Job job = Job.getInstance();
-    Set<Pair<Text,Text>> cols = new HashSet<>();
-    cols.add(new Pair<>(new Text(""), null));
-    cols.add(new Pair<>(new Text("foo"), new Text("bar")));
-    cols.add(new Pair<>(new Text(""), new Text("bar")));
-    cols.add(new Pair<>(new Text(""), new Text("")));
-    cols.add(new Pair<>(new Text("foo"), new Text("")));
-    AccumuloInputFormat.fetchColumns(job, cols);
-    Set<Pair<Text,Text>> setCols = AccumuloInputFormat.getFetchedColumns(job);
-    assertEquals(cols, setCols);
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
deleted file mode 100644
index 0ad8c05..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ /dev/null
@@ -1,63 +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.junit.jupiter.api.Assertions.assertEquals;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.WithTestNames;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormatTest extends WithTestNames {
-
-  /**
-   * Verify {@link InputTableConfig} objects get correctly serialized in the JobContext.
-   */
-  @Test
-  public void testInputTableConfigSerialization() throws IOException {
-    String table1 = testName() + "1";
-    String table2 = testName() + "2";
-    Job job = Job.getInstance();
-
-    InputTableConfig tableConfig = new InputTableConfig()
-        .setRanges(Collections.singletonList(new Range("a", "b")))
-        .fetchColumns(Collections.singleton(new Pair<>(new Text("CF1"), new Text("CQ1"))))
-        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
-
-    Map<String,InputTableConfig> configMap = new HashMap<>();
-    configMap.put(table1, tableConfig);
-    configMap.put(table2, tableConfig);
-
-    AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
-
-    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, table1));
-    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, table2));
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
deleted file mode 100644
index aa24a86..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
+++ /dev/null
@@ -1,75 +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.concurrent.TimeUnit.MILLISECONDS;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormatTest {
-
-  @Test
-  public void testBWSettings() throws IOException {
-    Job job = Job.getInstance();
-
-    // make sure we aren't testing defaults
-    final BatchWriterConfig bwDefaults = new BatchWriterConfig();
-    assertNotEquals(7654321L, bwDefaults.getMaxLatency(MILLISECONDS));
-    assertNotEquals(9898989L, bwDefaults.getTimeout(MILLISECONDS));
-    assertNotEquals(42, bwDefaults.getMaxWriteThreads());
-    assertNotEquals(1123581321L, bwDefaults.getMaxMemory());
-
-    final BatchWriterConfig bwConfig = new BatchWriterConfig();
-    bwConfig.setMaxLatency(7654321L, MILLISECONDS);
-    bwConfig.setTimeout(9898989L, MILLISECONDS);
-    bwConfig.setMaxWriteThreads(42);
-    bwConfig.setMaxMemory(1123581321L);
-    AccumuloOutputFormat.setBatchWriterOptions(job, bwConfig);
-
-    AccumuloOutputFormat myAOF = new AccumuloOutputFormat() {
-      @Override
-      public void checkOutputSpecs(JobContext job) {
-        BatchWriterConfig bwOpts = getBatchWriterOptions(job);
-
-        // passive check
-        assertEquals(bwConfig.getMaxLatency(MILLISECONDS), bwOpts.getMaxLatency(MILLISECONDS));
-        assertEquals(bwConfig.getTimeout(MILLISECONDS), bwOpts.getTimeout(MILLISECONDS));
-        assertEquals(bwConfig.getMaxWriteThreads(), bwOpts.getMaxWriteThreads());
-        assertEquals(bwConfig.getMaxMemory(), bwOpts.getMaxMemory());
-
-        // explicit check
-        assertEquals(7654321L, bwOpts.getMaxLatency(MILLISECONDS));
-        assertEquals(9898989L, bwOpts.getTimeout(MILLISECONDS));
-        assertEquals(42, bwOpts.getMaxWriteThreads());
-        assertEquals(1123581321L, bwOpts.getMaxMemory());
-
-      }
-    };
-    myAOF.checkOutputSpecs(job);
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
deleted file mode 100644
index 8a27d54..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
+++ /dev/null
@@ -1,130 +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.junit.jupiter.api.Assertions.assertEquals;
-
-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.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.data.Range;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class InputTableConfigTest {
-
-  private InputTableConfig tableQueryConfig;
-
-  @BeforeEach
-  public void setUp() {
-    tableQueryConfig = new InputTableConfig();
-  }
-
-  @Test
-  public void testSerialization_OnlyTable() throws IOException {
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-
-    assertEquals(tableQueryConfig, actualConfig);
-  }
-
-  @Test
-  public void testSerialization_allBooleans() throws IOException {
-    tableQueryConfig.setAutoAdjustRanges(false);
-    tableQueryConfig.setOfflineScan(true);
-    tableQueryConfig.setUseIsolatedScanners(true);
-    tableQueryConfig.setUseLocalIterators(true);
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-
-    assertEquals(tableQueryConfig, actualConfig);
-  }
-
-  @Test
-  public void testSerialization_ranges() throws IOException {
-    List<Range> ranges = new ArrayList<>();
-    ranges.add(new Range("a", "b"));
-    ranges.add(new Range("c", "d"));
-    tableQueryConfig.setRanges(ranges);
-
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-
-    assertEquals(ranges, actualConfig.getRanges());
-  }
-
-  @Test
-  public void testSerialization_columns() throws IOException {
-    Set<Pair<Text,Text>> columns = new HashSet<>();
-    columns.add(new Pair<>(new Text("cf1"), new Text("cq1")));
-    columns.add(new Pair<>(new Text("cf2"), null));
-    tableQueryConfig.fetchColumns(columns);
-
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-
-    assertEquals(actualConfig.getFetchedColumns(), columns);
-  }
-
-  @Test
-  public void testSerialization_iterators() throws IOException {
-    List<IteratorSetting> settings = new ArrayList<>();
-    settings.add(new IteratorSetting(50, "iter", "iterclass"));
-    settings.add(new IteratorSetting(55, "iter2", "iterclass2"));
-    tableQueryConfig.setIterators(settings);
-    byte[] serialized = serialize(tableQueryConfig);
-    InputTableConfig actualConfig = deserialize(serialized);
-    assertEquals(actualConfig.getIterators(), settings);
-
-  }
-
-  @Test
-  public void testSamplerConfig() throws IOException {
-    SamplerConfiguration sc = new SamplerConfiguration("com.foo.S1").addOption("k1", "v1");
-    tableQueryConfig.setSamplerConfiguration(sc);
-    InputTableConfig actualConfig = deserialize(serialize(tableQueryConfig));
-    assertEquals(sc, actualConfig.getSamplerConfiguration());
-  }
-
-  private byte[] serialize(InputTableConfig tableQueryConfig) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    tableQueryConfig.write(new DataOutputStream(baos));
-    baos.close();
-    return baos.toByteArray();
-  }
-
-  private InputTableConfig deserialize(byte[] bytes) throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-    InputTableConfig actualConfig = new InputTableConfig(new DataInputStream(bais));
-    bais.close();
-    return actualConfig;
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
deleted file mode 100644
index e608ff1..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
+++ /dev/null
@@ -1,119 +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.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-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.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.iterators.user.SummingCombiner;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class RangeInputSplitTest {
-
-  @Test
-  public void testSimpleWritable() throws IOException {
-    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new Key("a"), new Key("b")),
-        new String[] {"localhost"});
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-
-    RangeInputSplit newSplit = new RangeInputSplit();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-
-    assertEquals(split.getTableName(), newSplit.getTableName());
-    assertEquals(split.getTableId(), newSplit.getTableId());
-    assertEquals(split.getRange(), newSplit.getRange());
-    assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
-  }
-
-  @Test
-  public void testAllFieldsWritable() throws IOException {
-    RangeInputSplit split = new RangeInputSplit("table", "1", new Range(new Key("a"), new Key("b")),
-        new String[] {"localhost"});
-
-    Set<Pair<Text,Text>> fetchedColumns = new HashSet<>();
-
-    fetchedColumns.add(new Pair<>(new Text("colf1"), new Text("colq1")));
-    fetchedColumns.add(new Pair<>(new Text("colf2"), new Text("colq2")));
-
-    // Fake some iterators
-    ArrayList<IteratorSetting> iterators = new ArrayList<>();
-    IteratorSetting setting = new IteratorSetting(50, SummingCombiner.class);
-    setting.addOption("foo", "bar");
-    iterators.add(setting);
-
-    setting = new IteratorSetting(100, WholeRowIterator.class);
-    setting.addOption("bar", "foo");
-    iterators.add(setting);
-
-    split.setTableName("table");
-    split.setOffline(true);
-    split.setIsolatedScan(true);
-    split.setUsesLocalIterators(true);
-    split.setFetchedColumns(fetchedColumns);
-    split.setIterators(iterators);
-    split.setLogLevel(Level.WARN);
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-
-    RangeInputSplit newSplit = new RangeInputSplit();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-
-    assertEquals(split.getRange(), newSplit.getRange());
-    assertArrayEquals(split.getLocations(), newSplit.getLocations());
-
-    assertEquals(split.getTableName(), newSplit.getTableName());
-    assertEquals(split.isOffline(), newSplit.isOffline());
-    assertEquals(split.isIsolatedScan(), newSplit.isOffline());
-    assertEquals(split.usesLocalIterators(), newSplit.usesLocalIterators());
-    assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
-    assertEquals(split.getIterators(), newSplit.getIterators());
-    assertEquals(split.getLogLevel(), newSplit.getLogLevel());
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
deleted file mode 100644
index c487149..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.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.lib.partition;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class RangePartitionerTest {
-
-  private static Text[] cutArray = {new Text("A"), new Text("B"), new Text("C")};
-
-  @Test
-  public void testNoSubBins() throws IOException {
-    for (int i = -2; i < 2; ++i) {
-      checkExpectedBins(i, new String[] {"A", "B", "C"}, new int[] {0, 1, 2});
-      checkExpectedBins(i, new String[] {"C", "A", "B"}, new int[] {2, 0, 1});
-      checkExpectedBins(i, new String[] {"", "AA", "BB", "CC"}, new int[] {0, 1, 2, 3});
-    }
-  }
-
-  @Test
-  public void testSubBins() throws IOException {
-    checkExpectedRangeBins(2, new String[] {"A", "B", "C"}, new int[] {1, 3, 5});
-    checkExpectedRangeBins(2, new String[] {"C", "A", "B"}, new int[] {5, 1, 3});
-    checkExpectedRangeBins(2, new String[] {"", "AA", "BB", "CC"}, new int[] {1, 3, 5, 7});
-
-    checkExpectedRangeBins(3, new String[] {"A", "B", "C"}, new int[] {2, 5, 8});
-    checkExpectedRangeBins(3, new String[] {"C", "A", "B"}, new int[] {8, 2, 5});
-    checkExpectedRangeBins(3, new String[] {"", "AA", "BB", "CC"}, new int[] {2, 5, 8, 11});
-
-    checkExpectedRangeBins(10, new String[] {"A", "B", "C"}, new int[] {9, 19, 29});
-    checkExpectedRangeBins(10, new String[] {"C", "A", "B"}, new int[] {29, 9, 19});
-    checkExpectedRangeBins(10, new String[] {"", "AA", "BB", "CC"}, new int[] {9, 19, 29, 39});
-  }
-
-  private RangePartitioner prepPartitioner(int numSubBins) throws IOException {
-    Job job = Job.getInstance();
-    RangePartitioner.setNumSubBins(job, numSubBins);
-    RangePartitioner rp = new RangePartitioner();
-    rp.setConf(job.getConfiguration());
-    return rp;
-  }
-
-  private void checkExpectedRangeBins(int numSubBins, String[] strings, int[] rangeEnds)
-      throws IOException {
-    assertEquals(strings.length, rangeEnds.length);
-    for (int i = 0; i < strings.length; ++i) {
-      int endRange = rangeEnds[i];
-      int startRange = endRange + 1 - numSubBins;
-      int part =
-          prepPartitioner(numSubBins).findPartition(new Text(strings[i]), cutArray, numSubBins);
-      assertTrue(part >= startRange);
-      assertTrue(part <= endRange);
-    }
-  }
-
-  private void checkExpectedBins(int numSubBins, String[] strings, int[] bins) throws IOException {
-    assertEquals(strings.length, bins.length);
-    for (int i = 0; i < strings.length; ++i) {
-      int bin = bins[i], part =
-          prepPartitioner(numSubBins).findPartition(new Text(strings[i]), cutArray, numSubBins);
-      assertEquals(bin, part);
-    }
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java
index ec07de4..fcf2f62 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/rfile/RFileClientTest.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.client.rfile;
 
 import static com.google.common.collect.MoreCollectors.onlyElement;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -27,7 +28,6 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -44,6 +44,7 @@
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.rfile.RFile.InputArguments.FencedPath;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.summary.CounterSummary;
@@ -56,27 +57,29 @@
 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.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.file.rfile.RFile.RFileSKVIterator;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
+@SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path is set by test, not user")
 public class RFileClientTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path is set by test, not user")
   private String createTmpTestFile() throws IOException {
     File dir = new File(System.getProperty("user.dir") + "/target/rfile-test");
     assertTrue(dir.mkdirs() || dir.isDirectory());
@@ -188,6 +191,16 @@
     return map;
   }
 
+  SortedMap<Key,Value> toMap(FileSKVIterator iterator) throws IOException {
+    TreeMap<Key,Value> map = new TreeMap<>();
+    while (iterator.hasTop()) {
+      // Need to copy Value as the reference gets reused
+      map.put(iterator.getTopKey(), new Value(iterator.getTopValue()));
+      iterator.next();
+    }
+    return map;
+  }
+
   @Test
   public void testMultipleSources() throws Exception {
     SortedMap<Key,Value> testData1 = createTestData(10, 10, 10);
@@ -212,6 +225,87 @@
   }
 
   @Test
+  public void testFencingScanner() throws Exception {
+    SortedMap<Key,Value> testData = createTestData(10, 10, 10);
+
+    String testFile = createRFile(testData);
+
+    LocalFileSystem localFs = FileSystem.getLocal(new Configuration());
+
+    Range range = new Range(rowStr(3), true, rowStr(14), true);
+    Scanner scanner =
+        RFile.newScanner().from(new FencedPath(new Path(new File(testFile).toURI()), range))
+            .withFileSystem(localFs).build();
+
+    TreeMap<Key,Value> expected = new TreeMap<>(testData);
+
+    // Range is set on the RFile iterator itself and not the scanner
+    assertEquals(expected.subMap(range.getStartKey(), range.getEndKey()), toMap(scanner));
+
+    scanner.close();
+  }
+
+  @Test
+  public void testRequiresRowRange() throws Exception {
+    SortedMap<Key,Value> testData = createTestData(10, 10, 10);
+    String testFile = createRFile(testData);
+
+    // Row Ranges may have null for start and/or end row or be set.
+    // If start is set, it must be inclusive and if end is set it ust be exclusive.
+    // End key must also be an exclusive key (end in 0x00 byte).
+    // Lastly only the row portion of a key is allowed.
+
+    // Test valid Row Ranges
+    new FencedPath(new Path(new File(testFile).toURI()), new Range());
+    // This constructor converts to the proper inclusive/exclusive rows
+    new FencedPath(new Path(new File(testFile).toURI()),
+        new Range(rowStr(3), true, rowStr(14), true));
+    new FencedPath(new Path(new File(testFile).toURI()), new Range(new Key(rowStr(3)), true,
+        new Key(rowStr(14)).followingKey(PartialKey.ROW), false));
+
+    // Test invalid Row Ranges
+    // Missing 0x00 byte
+    assertThrows(IllegalArgumentException.class,
+        () -> new FencedPath(new Path(new File(testFile).toURI()),
+            new Range(new Key(rowStr(3)), true, new Key(rowStr(14)), false)));
+    // End key inclusive
+    assertThrows(IllegalArgumentException.class,
+        () -> new FencedPath(new Path(new File(testFile).toURI()),
+            new Range(new Key(rowStr(3)), true, new Key(rowStr(14)), true)));
+    // Start key exclusive
+    assertThrows(IllegalArgumentException.class,
+        () -> new FencedPath(new Path(new File(testFile).toURI()),
+            new Range(new Key(rowStr(3)), false, new Key(rowStr(14)), false)));
+    // CF is set which is not allowed
+    assertThrows(IllegalArgumentException.class,
+        () -> new FencedPath(new Path(new File(testFile).toURI()),
+            new Range(new Key(rowStr(3), colStr(3)), true,
+                new Key(rowStr(14)).followingKey(PartialKey.ROW), false)));
+  }
+
+  @Test
+  public void testFencingReader() throws Exception {
+    SortedMap<Key,Value> testData = createTestData(10, 10, 10);
+
+    String testFile = createRFile(testData);
+
+    LocalFileSystem localFs = FileSystem.getLocal(new Configuration());
+
+    Range range = new Range(rowStr(3), true, rowStr(14), true);
+
+    RFileSKVIterator reader =
+        getReader(localFs, UnreferencedTabletFile.ofRanged(localFs, new File(testFile), range));
+    reader.seek(new Range(), List.of(), false);
+
+    TreeMap<Key,Value> expected = new TreeMap<>(testData);
+
+    // Range is set on the RFile iterator itself and not the scanner
+    assertEquals(expected.subMap(range.getStartKey(), range.getEndKey()), toMap(reader));
+
+    reader.close();
+  }
+
+  @Test
   public void testWriterTableProperties() throws Exception {
     LocalFileSystem localFs = FileSystem.getLocal(new Configuration());
 
@@ -227,7 +321,8 @@
     writer.append(testData1.entrySet());
     writer.close();
 
-    Reader reader = getReader(localFs, testFile);
+    RFileSKVIterator reader =
+        getReader(localFs, UnreferencedTabletFile.of(localFs, new File(testFile)));
     FileSKVIterator iiter = reader.getIndex();
 
     int count = 0;
@@ -289,7 +384,8 @@
 
     scanner.close();
 
-    Reader reader = getReader(localFs, testFile);
+    Reader reader =
+        (Reader) getReader(localFs, UnreferencedTabletFile.of(localFs, new File(testFile)));
     Map<String,ArrayList<ByteSequence>> lGroups = reader.getLocalityGroupCF();
     assertTrue(lGroups.containsKey("z"));
     assertEquals(2, lGroups.get("z").size());
@@ -520,7 +616,7 @@
     Scanner scanner = RFile.newScanner().from(testFile).withFileSystem(localFs)
         .withIndexCache(1000000).withDataCache(10000000).build();
 
-    random.ints(100, 0, 10_000).forEach(r -> {
+    RANDOM.get().ints(100, 0, 10_000).forEach(r -> {
       scanner.setRange(new Range(rowStr(r)));
       String actual = scanner.stream().collect(onlyElement()).getKey().getRow().toString();
       assertEquals(rowStr(r), actual);
@@ -810,8 +906,9 @@
     }
   }
 
-  private Reader getReader(LocalFileSystem localFs, String testFile) throws IOException {
-    return (Reader) FileOperations.getInstance().newReaderBuilder()
+  private RFileSKVIterator getReader(LocalFileSystem localFs, UnreferencedTabletFile testFile)
+      throws IOException {
+    return (RFileSKVIterator) FileOperations.getInstance().newReaderBuilder()
         .forFile(testFile, localFs, localFs.getConf(), NoCryptoServiceFactory.NONE)
         .withTableConfiguration(DefaultConfiguration.getInstance()).build();
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java
index b94bbba..1303fbd 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientConfConverterTest.java
@@ -35,18 +35,39 @@
   @Test
   public void testBasic() {
     Properties before = new Properties();
+
+    // this will be dropped when converting to AccumuloConfiguration
     before.setProperty(ClientProperty.INSTANCE_NAME.getKey(), "instance");
-    before.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), "zookeepers");
     ClientProperty.setPassword(before, "mypass");
-    before.setProperty(ClientProperty.SSL_ENABLED.getKey(), "true");
-    before.setProperty(ClientProperty.SSL_KEYSTORE_PATH.getKey(), "key_path");
-    before.setProperty(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(), "key_pass");
-    before.setProperty(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(), "trust_path");
-    before.setProperty(ClientProperty.SASL_ENABLED.getKey(), "true");
-    before.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(), "primary");
     before.setProperty(ClientProperty.BATCH_WRITER_THREADS_MAX.getKey(), "5");
 
-    Properties after = ClientConfConverter.toProperties(ClientConfConverter.toClientConf(before));
+    // these will map to equivalent in AccumuloConfiguration
+    before.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), "zookeepers");
+    before.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(), "20s");
+    before.setProperty(ClientProperty.SSL_ENABLED.getKey(), "true");
+    before.setProperty(ClientProperty.SSL_USE_JSSE.getKey(), "true");
+    before.setProperty(ClientProperty.SSL_KEYSTORE_PATH.getKey(), "key_path");
+    before.setProperty(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(), "key_pass");
+    before.setProperty(ClientProperty.SSL_KEYSTORE_TYPE.getKey(), "jks");
+    before.setProperty(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(), "trust_path");
+    before.setProperty(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(), "trust_pass");
+    before.setProperty(ClientProperty.SSL_TRUSTSTORE_TYPE.getKey(), "jks");
+    before.setProperty(ClientProperty.SASL_ENABLED.getKey(), "true");
+    before.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(), "primary");
+    before.setProperty(ClientProperty.SASL_QOP.getKey(), "auth-int");
+
+    Properties after = ClientConfConverter.toProperties(ClientConfConverter.toAccumuloConf(before));
+
+    // some props don't have an equivalent in the AccumuloConfiguration; set them here and check
+    assertNotEquals(before, after);
+    ClientProperty.setPassword(after, "mypass");
+
+    assertNotEquals(before, after);
+    after.setProperty(ClientProperty.BATCH_WRITER_THREADS_MAX.getKey(), "5");
+
+    assertNotEquals(before, after);
+    after.setProperty(ClientProperty.INSTANCE_NAME.getKey(), "instance");
+
     assertEquals(before, after);
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TableOperationsHelperTest.java
index 63337b4..7b89c9a 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TableOperationsHelperTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/TableOperationsHelperTest.java
@@ -188,13 +188,7 @@
     }
 
     @Override
-    @Deprecated(since = "2.0.0")
-    public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) {}
-
-    @Override
-    public void offline(String tableName) {
-
-    }
+    public void offline(String tableName) {}
 
     @Override
     public boolean isOnline(String tableName) {
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
index ea20ee6..242cc18 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/TabletLocatorImplTest.java
@@ -50,8 +50,8 @@
 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.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.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -64,7 +64,7 @@
 
   private static final KeyExtent ROOT_TABLE_EXTENT = RootTable.EXTENT;
   private static final KeyExtent METADATA_TABLE_EXTENT =
-      new KeyExtent(MetadataTable.ID, null, ROOT_TABLE_EXTENT.endRow());
+      new KeyExtent(AccumuloTable.METADATA.tableId(), null, ROOT_TABLE_EXTENT.endRow());
 
   static KeyExtent createNewKeyExtent(String table, String endRow, String prevEndRow) {
     return new KeyExtent(TableId.of(table), endRow == null ? null : new Text(endRow),
@@ -162,7 +162,7 @@
 
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache =
-        new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo, new YesLockChecker());
+        new TabletLocatorImpl(AccumuloTable.METADATA.tableId(), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab1TabletCache =
         new TabletLocatorImpl(TableId.of(table), rootTabletCache, ttlo, tslc);
 
@@ -170,7 +170,7 @@
 
     for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
       setLocation(tservers, metaTabLoc, METADATA_TABLE_EXTENT, entry.getKey(),
-          entry.getValue().tablet_location);
+          entry.getValue().getTserverLocation());
     }
 
     return tab1TabletCache;
@@ -236,7 +236,7 @@
 
     HashSet<KeyExtent> eic = new HashSet<>();
     for (TabletLocation tl : metaCache.values()) {
-      eic.add(tl.tablet_extent);
+      eic.add(tl.getExtent());
     }
 
     assertEquals(expected, eic);
@@ -485,22 +485,22 @@
     public TabletLocations lookupTablet(ClientContext context, TabletLocation src, Text row,
         Text stopRow, TabletLocator parent) {
 
-      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(src.tablet_location);
+      Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(src.getTserverLocation());
 
       if (tablets == null) {
-        parent.invalidateCache(context, src.tablet_location);
+        parent.invalidateCache(context, src.getTserverLocation());
         return null;
       }
 
-      SortedMap<Key,Value> tabletData = tablets.get(src.tablet_extent);
+      SortedMap<Key,Value> tabletData = tablets.get(src.getExtent());
 
       if (tabletData == null) {
-        parent.invalidateCache(src.tablet_extent);
+        parent.invalidateCache(src.getExtent());
         return null;
       }
 
       // the following clip is done on a tablet, do it here to see if it throws exceptions
-      src.tablet_extent.toDataRange().clip(new Range(row, true, stopRow, true));
+      src.getExtent().toDataRange().clip(new Range(row, true, stopRow, true));
 
       Key startKey = new Key(row);
       Key stopKey = new Key(stopRow).followingKey(PartialKey.ROW);
@@ -596,7 +596,7 @@
         tservers.tservers.computeIfAbsent(server, k -> new HashMap<>());
     SortedMap<Key,Value> tabletData = tablets.computeIfAbsent(tablet, k -> new TreeMap<>());
     if (!tabletData.isEmpty()) {
-      throw new RuntimeException("Asked for empty tablet, but non empty tablet exists");
+      throw new IllegalStateException("Asked for empty tablet, but non empty tablet exists");
     }
   }
 
@@ -661,8 +661,8 @@
       assertNull(tl);
     } else {
       assertNotNull(tl);
-      assertEquals(server, tl.tablet_location);
-      assertEquals(expected, tl.tablet_extent);
+      assertEquals(server, tl.getTserverLocation());
+      assertEquals(expected, tl.getExtent());
     }
   }
 
@@ -678,7 +678,7 @@
 
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache =
-        new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo, new YesLockChecker());
+        new TabletLocatorImpl(AccumuloTable.METADATA.tableId(), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab1TabletCache =
         new TabletLocatorImpl(TableId.of("tab1"), rootTabletCache, ttlo, new YesLockChecker());
 
@@ -763,9 +763,9 @@
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3");
 
     // simulate the metadata table splitting
-    KeyExtent mte1 =
-        new KeyExtent(MetadataTable.ID, tab1e21.toMetaRow(), ROOT_TABLE_EXTENT.endRow());
-    KeyExtent mte2 = new KeyExtent(MetadataTable.ID, null, tab1e21.toMetaRow());
+    KeyExtent mte1 = new KeyExtent(AccumuloTable.METADATA.tableId(), tab1e21.toMetaRow(),
+        ROOT_TABLE_EXTENT.endRow());
+    KeyExtent mte2 = new KeyExtent(AccumuloTable.METADATA.tableId(), null, tab1e21.toMetaRow());
 
     setLocation(tservers, "tserver4", ROOT_TABLE_EXTENT, mte1, "tserver5");
     setLocation(tservers, "tserver4", ROOT_TABLE_EXTENT, mte2, "tserver6");
@@ -803,9 +803,10 @@
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9");
 
     // simulate a hole in the metadata, caused by a partial split
-    KeyExtent mte11 =
-        new KeyExtent(MetadataTable.ID, tab1e1.toMetaRow(), ROOT_TABLE_EXTENT.endRow());
-    KeyExtent mte12 = new KeyExtent(MetadataTable.ID, tab1e21.toMetaRow(), tab1e1.toMetaRow());
+    KeyExtent mte11 = new KeyExtent(AccumuloTable.METADATA.tableId(), tab1e1.toMetaRow(),
+        ROOT_TABLE_EXTENT.endRow());
+    KeyExtent mte12 =
+        new KeyExtent(AccumuloTable.METADATA.tableId(), tab1e21.toMetaRow(), tab1e1.toMetaRow());
     deleteServer(tservers, "tserver10");
     setLocation(tservers, "tserver4", ROOT_TABLE_EXTENT, mte12, "tserver10");
     setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
@@ -1410,15 +1411,16 @@
   @Test
   public void testBug1() throws Exception {
     // a bug that occurred while running continuous ingest
-    KeyExtent mte1 = new KeyExtent(MetadataTable.ID, new Text("0;0bc"), ROOT_TABLE_EXTENT.endRow());
-    KeyExtent mte2 = new KeyExtent(MetadataTable.ID, null, new Text("0;0bc"));
+    KeyExtent mte1 = new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("0;0bc"),
+        ROOT_TABLE_EXTENT.endRow());
+    KeyExtent mte2 = new KeyExtent(AccumuloTable.METADATA.tableId(), null, new Text("0;0bc"));
 
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache =
-        new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo, new YesLockChecker());
+        new TabletLocatorImpl(AccumuloTable.METADATA.tableId(), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab0TabletCache =
         new TabletLocatorImpl(TableId.of("0"), rootTabletCache, ttlo, new YesLockChecker());
 
@@ -1439,15 +1441,16 @@
   @Test
   public void testBug2() throws Exception {
     // a bug that occurred while running a functional test
-    KeyExtent mte1 = new KeyExtent(MetadataTable.ID, new Text("~"), ROOT_TABLE_EXTENT.endRow());
-    KeyExtent mte2 = new KeyExtent(MetadataTable.ID, null, new Text("~"));
+    KeyExtent mte1 =
+        new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("~"), ROOT_TABLE_EXTENT.endRow());
+    KeyExtent mte2 = new KeyExtent(AccumuloTable.METADATA.tableId(), null, new Text("~"));
 
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
 
     RootTabletLocator rtl = new TestRootTabletLocator();
     TabletLocatorImpl rootTabletCache =
-        new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo, new YesLockChecker());
+        new TabletLocatorImpl(AccumuloTable.METADATA.tableId(), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab0TabletCache =
         new TabletLocatorImpl(TableId.of("0"), rootTabletCache, ttlo, new YesLockChecker());
 
@@ -1467,11 +1470,15 @@
   // being merged away, caused locating tablets to fail
   @Test
   public void testBug3() throws Exception {
-    KeyExtent mte1 = new KeyExtent(MetadataTable.ID, new Text("1;c"), ROOT_TABLE_EXTENT.endRow());
-    KeyExtent mte2 = new KeyExtent(MetadataTable.ID, new Text("1;f"), new Text("1;c"));
-    KeyExtent mte3 = new KeyExtent(MetadataTable.ID, new Text("1;j"), new Text("1;f"));
-    KeyExtent mte4 = new KeyExtent(MetadataTable.ID, new Text("1;r"), new Text("1;j"));
-    KeyExtent mte5 = new KeyExtent(MetadataTable.ID, null, new Text("1;r"));
+    KeyExtent mte1 = new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("1;c"),
+        ROOT_TABLE_EXTENT.endRow());
+    KeyExtent mte2 =
+        new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("1;f"), new Text("1;c"));
+    KeyExtent mte3 =
+        new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("1;j"), new Text("1;f"));
+    KeyExtent mte4 =
+        new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("1;r"), new Text("1;j"));
+    KeyExtent mte5 = new KeyExtent(AccumuloTable.METADATA.tableId(), null, new Text("1;r"));
 
     KeyExtent ke1 = new KeyExtent(TableId.of("1"), null, null);
 
@@ -1481,7 +1488,7 @@
     RootTabletLocator rtl = new TestRootTabletLocator();
 
     TabletLocatorImpl rootTabletCache =
-        new TabletLocatorImpl(MetadataTable.ID, rtl, ttlo, new YesLockChecker());
+        new TabletLocatorImpl(AccumuloTable.METADATA.tableId(), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab0TabletCache =
         new TabletLocatorImpl(TableId.of("1"), rootTabletCache, ttlo, new YesLockChecker());
 
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java
index bc9e7cb..44fd571 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ThriftTransportKeyTest.java
@@ -36,7 +36,6 @@
 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 org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -44,6 +43,8 @@
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class ThriftTransportKeyTest {
 
   private static final String primary = "accumulo";
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java
index bb59ff4..5a1ac90 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCacheTest.java
@@ -18,11 +18,10 @@
  */
 package org.apache.accumulo.core.clientImpl.bulk;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -36,12 +35,8 @@
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.util.concurrent.Uninterruptibles;
-
 public class ConcurrentKeyExtentCacheTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
   private static List<KeyExtent> extents = new ArrayList<>();
   private static Set<KeyExtent> extentsSet = new HashSet<>();
 
@@ -83,8 +78,11 @@
         }
       }
 
-      Uninterruptibles.sleepUninterruptibly(3, MILLISECONDS);
-
+      try {
+        Thread.sleep(3);
+      } catch (InterruptedException ex) {
+        // ignore exception
+      }
       return extents.subList(index, extents.size()).stream().limit(73);
     }
   }
@@ -100,13 +98,13 @@
 
     TestCache tc = new TestCache();
 
-    random.ints(20000, 0, 256).mapToObj(i -> new Text(String.format("%02x", i))).sequential()
+    RANDOM.get().ints(20000, 0, 256).mapToObj(i -> new Text(String.format("%02x", i))).sequential()
         .forEach(lookupRow -> testLookup(tc, lookupRow));
     assertEquals(extentsSet, tc.seen);
 
     // try parallel
     TestCache tc2 = new TestCache();
-    random.ints(20000, 0, 256).mapToObj(i -> new Text(String.format("%02x", i))).parallel()
+    RANDOM.get().ints(20000, 0, 256).mapToObj(i -> new Text(String.format("%02x", i))).parallel()
         .forEach(lookupRow -> testLookup(tc2, lookupRow));
     assertEquals(extentsSet, tc.seen);
   }
@@ -115,13 +113,13 @@
   public void testRandom() {
     TestCache tc = new TestCache();
 
-    random.ints(20000).mapToObj(i -> new Text(String.format("%08x", i))).sequential()
+    RANDOM.get().ints(20000).mapToObj(i -> new Text(String.format("%08x", i))).sequential()
         .forEach(lookupRow -> testLookup(tc, lookupRow));
     assertEquals(extentsSet, tc.seen);
 
     // try parallel
     TestCache tc2 = new TestCache();
-    random.ints(20000).mapToObj(i -> new Text(String.format("%08x", i))).parallel()
+    RANDOM.get().ints(20000).mapToObj(i -> new Text(String.format("%08x", i))).parallel()
         .forEach(lookupRow -> testLookup(tc2, lookupRow));
     assertEquals(extentsSet, tc2.seen);
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
deleted file mode 100644
index e44414c..0000000
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
+++ /dev/null
@@ -1,117 +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 org.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-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.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-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.iterators.user.SummingCombiner;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.0.0")
-public class BatchInputSplitTest {
-
-  @Test
-  public void testSimpleWritable() throws IOException {
-    List<Range> ranges = Collections.singletonList(new Range(new Key("a"), new Key("b")));
-    BatchInputSplit split =
-        new BatchInputSplit("table", TableId.of("1"), ranges, new String[] {"localhost"});
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-
-    BatchInputSplit newSplit = new BatchInputSplit();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-
-    assertEquals(split.getTableName(), newSplit.getTableName());
-    assertEquals(split.getTableId(), newSplit.getTableId());
-    assertEquals(split.getRanges(), newSplit.getRanges());
-    assertTrue(Arrays.equals(split.getLocations(), newSplit.getLocations()));
-  }
-
-  @Test
-  public void testAllFieldsWritable() throws IOException {
-    List<Range> ranges = Collections.singletonList(new Range(new Key("a"), new Key("b")));
-    BatchInputSplit split =
-        new BatchInputSplit("table", TableId.of("1"), ranges, new String[] {"localhost"});
-
-    Set<Pair<Text,Text>> fetchedColumns = new HashSet<>();
-
-    fetchedColumns.add(new Pair<>(new Text("colf1"), new Text("colq1")));
-    fetchedColumns.add(new Pair<>(new Text("colf2"), new Text("colq2")));
-
-    // Fake some iterators
-    ArrayList<IteratorSetting> iterators = new ArrayList<>();
-    IteratorSetting setting = new IteratorSetting(50, SummingCombiner.class);
-    setting.addOption("foo", "bar");
-    iterators.add(setting);
-
-    setting = new IteratorSetting(100, WholeRowIterator.class);
-    setting.addOption("bar", "foo");
-    iterators.add(setting);
-
-    split.setTableName("table");
-    split.setFetchedColumns(fetchedColumns);
-    split.setIterators(iterators);
-    split.setLogLevel(Level.WARN);
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    split.write(dos);
-
-    BatchInputSplit newSplit = new BatchInputSplit();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream dis = new DataInputStream(bais);
-    newSplit.readFields(dis);
-
-    assertEquals(split.getRanges(), newSplit.getRanges());
-    assertArrayEquals(split.getLocations(), newSplit.getLocations());
-
-    assertEquals(split.getTableName(), newSplit.getTableName());
-    assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
-    assertEquals(split.getIterators(), newSplit.getIterators());
-    assertEquals(split.getLogLevel(), newSplit.getLogLevel());
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
deleted file mode 100644
index 05405ea..0000000
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
+++ /dev/null
@@ -1,131 +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 org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Base64;
-
-import org.apache.accumulo.core.Constants;
-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.PasswordToken;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.jupiter.api.Test;
-
-public class ConfiguratorBaseTest {
-
-  private enum PrivateTestingEnum {
-    SOMETHING, SOMETHING_ELSE
-  }
-
-  @Test
-  public void testEnumToConfKey() {
-    assertEquals(this.getClass().getSimpleName() + ".PrivateTestingEnum.Something",
-        ConfiguratorBase.enumToConfKey(this.getClass(), PrivateTestingEnum.SOMETHING));
-    assertEquals(this.getClass().getSimpleName() + ".PrivateTestingEnum.SomethingElse",
-        ConfiguratorBase.enumToConfKey(this.getClass(), PrivateTestingEnum.SOMETHING_ELSE));
-  }
-
-  @Test
-  public void testSetConnectorInfoClassOfQConfigurationStringAuthenticationToken() {
-    Configuration conf = new Configuration();
-    assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser",
-        new PasswordToken("testPassword"));
-    assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), conf));
-    AuthenticationToken token = ConfiguratorBase.getAuthenticationToken(this.getClass(), conf);
-    assertNotNull(token);
-    assertEquals(PasswordToken.class, token.getClass());
-    assertEquals(new PasswordToken("testPassword"), token);
-    assertEquals(
-        "inline:" + PasswordToken.class.getName() + ":"
-            + Base64.getEncoder().encodeToString(
-                AuthenticationTokenSerializer.serialize(new PasswordToken("testPassword"))),
-        conf.get(
-            ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.ConnectorInfo.TOKEN)));
-  }
-
-  @Test
-  public void testSetConnectorInfoClassOfQConfigurationStringString() {
-    Configuration conf = new Configuration();
-    assertFalse(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    ConfiguratorBase.setConnectorInfo(this.getClass(), conf, "testUser", "testFile");
-    assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
-    assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), conf));
-    assertEquals("file:testFile", conf.get(
-        ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.ConnectorInfo.TOKEN)));
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testSetZooKeeperInstance() {
-    Configuration conf = new Configuration();
-    ConfiguratorBase.setZooKeeperInstance(this.getClass(), conf,
-        org.apache.accumulo.core.client.ClientConfiguration.create()
-            .withInstance("testInstanceName").withZkHosts("testZooKeepers").withSsl(true)
-            .withZkTimeout(15000));
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        org.apache.accumulo.core.client.ClientConfiguration.deserialize(conf.get(ConfiguratorBase
-            .enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.CLIENT_CONFIG)));
-    assertEquals("testInstanceName", clientConf
-        .get(org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_NAME));
-    assertEquals("testZooKeepers", clientConf
-        .get(org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST));
-    assertEquals("true", clientConf.get(
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_ENABLED));
-    assertEquals("15000", clientConf.get(
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT));
-    assertEquals(org.apache.accumulo.core.client.ZooKeeperInstance.class.getSimpleName(), conf
-        .get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.TYPE)));
-  }
-
-  @Test
-  public void testSetLogLevel() {
-    Configuration conf = new Configuration();
-    Level currentLevel = Logger.getLogger(this.getClass()).getLevel();
-
-    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.DEBUG);
-    Logger.getLogger(this.getClass()).setLevel(currentLevel);
-    assertEquals(Level.DEBUG, ConfiguratorBase.getLogLevel(this.getClass(), conf));
-
-    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.INFO);
-    Logger.getLogger(this.getClass()).setLevel(currentLevel);
-    assertEquals(Level.INFO, ConfiguratorBase.getLogLevel(this.getClass(), conf));
-
-    ConfiguratorBase.setLogLevel(this.getClass(), conf, Level.FATAL);
-    Logger.getLogger(this.getClass()).setLevel(currentLevel);
-    assertEquals(Level.FATAL, ConfiguratorBase.getLogLevel(this.getClass(), conf));
-  }
-
-  @Test
-  public void testSetVisibilityCacheSize() {
-    Configuration conf = new Configuration();
-    assertEquals(Constants.DEFAULT_VISIBILITY_CACHE_SIZE,
-        ConfiguratorBase.getVisibilityCacheSize(conf));
-    ConfiguratorBase.setVisibilityCacheSize(conf, 2000);
-    assertEquals(2000, ConfiguratorBase.getVisibilityCacheSize(conf));
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java b/core/src/test/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurerTest.java
similarity index 87%
rename from server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java
rename to core/src/test/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurerTest.java
index 7567fbc..ef68da9 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/ConfigurableCompactionStrategyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/compaction/ShellCompactCommandConfigurerTest.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 static org.apache.accumulo.core.conf.ConfigurationTypeHelper.getFixedMemoryAsBytes;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -33,14 +33,12 @@
 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.CompactionConfigurer.Overrides;
-import org.apache.accumulo.core.compaction.CompactionSettings;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
 import org.junit.jupiter.api.Test;
 
-public class ConfigurableCompactionStrategyTest {
-
-  // file selection options are adequately tested by ShellServerIT
+public class ShellCompactCommandConfigurerTest {
 
   @Test
   public void testOutputOptions() throws URISyntaxException {
@@ -49,7 +47,7 @@
         .create(new URI("hdfs://nn1/accumulo/tables/1/t-009/F00001.rf"), 50000, 400));
 
     // test setting no output options
-    ConfigurableCompactionStrategy ccs = new ConfigurableCompactionStrategy();
+    ShellCompactCommandConfigurer ccs = new ShellCompactCommandConfigurer();
 
     Map<String,String> opts = new HashMap<>();
 
@@ -81,11 +79,21 @@
       }
 
       @Override
+      public TabletId getTabletId() {
+        return null;
+      }
+
+      @Override
       public Collection<CompactableFile> getInputFiles() {
         return files;
       }
 
       @Override
+      public Set<CompactableFile> getSelectedFiles() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
       public PluginEnvironment getEnvironment() {
         return null;
       }
@@ -96,7 +104,7 @@
     assertTrue(plan.getOverrides().isEmpty());
 
     // test setting all output options
-    ccs = new ConfigurableCompactionStrategy();
+    ccs = new ShellCompactCommandConfigurer();
 
     CompactionSettings.OUTPUT_BLOCK_SIZE_OPT.put(null, opts, "64K");
     CompactionSettings.OUTPUT_COMPRESSION_OPT.put(null, opts, "snappy");
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java b/core/src/test/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelectorTest.java
similarity index 70%
rename from core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
rename to core/src/test/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelectorTest.java
index d347979..ccbf295 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/ReplicationServicerThriftClient.java
+++ b/core/src/test/java/org/apache/accumulo/core/compaction/ShellCompactCommandSelectorTest.java
@@ -16,14 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.core.rpc.clients;
+package org.apache.accumulo.core.compaction;
 
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
+import java.net.URISyntaxException;
 
-public class ReplicationServicerThriftClient extends ThriftClientTypes<Client> {
+import org.junit.jupiter.api.Test;
 
-  ReplicationServicerThriftClient(String serviceName) {
-    super(serviceName, new Client.Factory());
+public class ShellCompactCommandSelectorTest {
+
+  @Test
+  public void testSelection() throws URISyntaxException {
+    // file selection options are adequately tested by ShellServerIT, so this is just a placeholder
   }
-
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/AccumuloConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/AccumuloConfigurationTest.java
index dc7a600..51c4930 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/AccumuloConfigurationTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/AccumuloConfigurationTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.conf;
 
+import static org.apache.accumulo.core.conf.Property.TABLE_ITERATOR_MINC_PREFIX;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
@@ -25,6 +26,7 @@
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
+import java.lang.reflect.Field;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
@@ -37,10 +39,6 @@
 
 public class AccumuloConfigurationTest {
 
-  @SuppressWarnings("removal")
-  private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
-      Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-
   @Test
   public void testGetPropertyByString() {
     AccumuloConfiguration c = DefaultConfiguration.getInstance();
@@ -235,8 +233,8 @@
     expected2.put(Property.TABLE_ITERATOR_SCAN_PREFIX.getKey() + "i1.opt", "o99");
     assertEquals(expected2, pm3);
 
-    Map<String,String> pm5 = tc.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
-    Map<String,String> pm6 = tc.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
+    Map<String,String> pm5 = tc.getAllPropertiesWithPrefix(TABLE_ITERATOR_MINC_PREFIX);
+    Map<String,String> pm6 = tc.getAllPropertiesWithPrefix(TABLE_ITERATOR_MINC_PREFIX);
     assertSame(pm5, pm6);
     assertEquals(0, pm5.size());
 
@@ -247,7 +245,7 @@
     Map<String,String> pm8 = tc.getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_SCAN_PREFIX);
     assertSame(pm3, pm8);
 
-    Map<String,String> pm9 = tc.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
+    Map<String,String> pm9 = tc.getAllPropertiesWithPrefix(TABLE_ITERATOR_MINC_PREFIX);
     assertSame(pm5, pm9);
 
     tc.set(Property.TABLE_ITERATOR_SCAN_PREFIX.getKey() + "i2", "class42");
@@ -268,13 +266,13 @@
     assertSame(pmC, pmD);
     assertEquals(expected1, pmC);
 
-    tc.set(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "ctx123", "hdfs://ib/p1");
+    tc.set(TABLE_ITERATOR_MINC_PREFIX.getKey() + "minc1", "abcd");
 
-    Map<String,String> pmE = tc.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
-    Map<String,String> pmF = tc.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
+    Map<String,String> pmE = tc.getAllPropertiesWithPrefix(TABLE_ITERATOR_MINC_PREFIX);
+    Map<String,String> pmF = tc.getAllPropertiesWithPrefix(TABLE_ITERATOR_MINC_PREFIX);
     assertSame(pmE, pmF);
     assertNotSame(pm5, pmE);
-    assertEquals(Map.of(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "ctx123", "hdfs://ib/p1"), pmE);
+    assertEquals(Map.of(TABLE_ITERATOR_MINC_PREFIX.getKey() + "minc1", "abcd"), pmE);
 
     Map<String,String> pmG = tc.getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_SCAN_PREFIX);
     Map<String,String> pmH = tc.getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_SCAN_PREFIX);
@@ -288,7 +286,7 @@
     assertSame(pmI, pmJ);
     assertEquals(expected1, pmI);
 
-    Map<String,String> pmK = tc.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
+    Map<String,String> pmK = tc.getAllPropertiesWithPrefix(TABLE_ITERATOR_MINC_PREFIX);
     assertSame(pmE, pmK);
 
     Map<String,String> pmL = tc.getAllPropertiesWithPrefix(Property.TABLE_ITERATOR_SCAN_PREFIX);
@@ -313,16 +311,7 @@
     assertTrue(sec.prioritizerClass.orElseThrow().isEmpty());
     assertTrue(sec.prioritizerOpts.isEmpty());
 
-    // ensure deprecated props is read if nothing else is set
-    tc.set("tserver.readahead.concurrent.max", "6");
-    assertEquals(6, sec.getCurrentMaxThreads());
-    assertEquals(Integer.parseInt(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getDefaultValue()),
-        sec.maxThreads);
-    ScanExecutorConfig sec2 = tc.getScanExecutors(false).stream()
-        .filter(c -> c.name.equals(defName)).findFirst().orElseThrow();
-    assertEquals(6, sec2.maxThreads);
-
-    // ensure new prop overrides deprecated prop
+    // ensure new props override default props
     tc.set(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey(), "9");
     assertEquals(9, sec.getCurrentMaxThreads());
     assertEquals(Integer.parseInt(Property.TSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getDefaultValue()),
@@ -339,7 +328,7 @@
     assertFalse(sec4.prioritizerClass.isPresent());
     assertTrue(sec4.prioritizerOpts.isEmpty());
 
-    tc.set("tserver.metadata.readahead.concurrent.max", "2");
+    tc.set(Property.TSERV_SCAN_EXECUTORS_META_THREADS.getKey(), "2");
     assertEquals(2, sec4.getCurrentMaxThreads());
     ScanExecutorConfig sec5 = tc.getScanExecutors(false).stream().filter(c -> c.name.equals("meta"))
         .findFirst().orElseThrow();
@@ -376,11 +365,12 @@
     assertEquals(44, sec8.maxThreads);
 
     // test scan server props
+    tc.set(Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS.getKey(), "6");
     Collection<ScanExecutorConfig> scanServExecutors = tc.getScanExecutors(true);
     assertEquals(2, scanServExecutors.size());
     ScanExecutorConfig sec9 =
         scanServExecutors.stream().filter(c -> c.name.equals(defName)).findFirst().orElseThrow();
-    // earlier in the test tserver.readahead.concurrent.max was set to 6
+    // verify set to 6
     assertEquals(6, sec9.maxThreads);
     assertFalse(sec9.priority.isPresent());
     assertTrue(sec9.prioritizerClass.orElseThrow().isEmpty());
@@ -393,43 +383,66 @@
   }
 
   // note: this is hard to test if there aren't any deprecated properties
-  // if that's the case, just comment this test out or create a dummy deprecated property
-  @SuppressWarnings("deprecation")
+  // Update a couple of non-deprecated properties using reflection for testing purposes
   @Test
-  public void testResolveDeprecated() {
+  public void testResolveDeprecated() throws Exception {
     var conf = new ConfigurationCopy();
 
-    // deprecated first argument
-    var e1 = assertThrows(IllegalArgumentException.class, () -> conf
-        .resolve(Property.INSTANCE_DFS_DIR, Property.INSTANCE_DFS_URI, Property.INSTANCE_DFS_URI));
-    assertEquals("Unexpected deprecated INSTANCE_DFS_DIR", e1.getMessage());
+    final Field isDeprecatedField =
+        Property.INSTANCE_ZK_HOST.getClass().getDeclaredField("isDeprecated");
+    isDeprecatedField.setAccessible(true);
 
-    // non-deprecated second argument
-    var e2 = assertThrows(IllegalArgumentException.class,
-        () -> conf.resolve(Property.INSTANCE_VOLUMES, Property.INSTANCE_DFS_DIR,
-            Property.INSTANCE_SECRET, Property.INSTANCE_DFS_DIR, Property.INSTANCE_VOLUMES));
-    assertEquals("Unexpected non-deprecated [INSTANCE_SECRET, INSTANCE_VOLUMES]", e2.getMessage());
+    // Capture the original setting. These are not deprecated but just in case they are in the
+    // future
+    // this will prevent the test from breaking when we reset at the end
+    final boolean origIsDepInstanceZkHost = Property.INSTANCE_ZK_HOST.isDeprecated();
+    final boolean origIsDepInstanceZkTimeout = Property.INSTANCE_ZK_TIMEOUT.isDeprecated();
 
-    // empty second argument always resolves to non-deprecated first argument
-    assertSame(Property.INSTANCE_VOLUMES, conf.resolve(Property.INSTANCE_VOLUMES));
+    try {
+      // Mark these 2 properties as deprecated just for testing purposes to make sure resolve works
+      isDeprecatedField.set(Property.INSTANCE_ZK_HOST, true);
+      isDeprecatedField.set(Property.INSTANCE_ZK_TIMEOUT, true);
 
-    // none are set, resolve to non-deprecated
-    assertSame(Property.INSTANCE_VOLUMES, conf.resolve(Property.INSTANCE_VOLUMES,
-        Property.INSTANCE_DFS_DIR, Property.INSTANCE_DFS_URI));
+      // deprecated first argument
+      var e1 =
+          assertThrows(IllegalArgumentException.class, () -> conf.resolve(Property.INSTANCE_ZK_HOST,
+              Property.INSTANCE_ZK_TIMEOUT, Property.INSTANCE_ZK_TIMEOUT));
+      assertEquals("Unexpected deprecated INSTANCE_ZK_HOST", e1.getMessage());
 
-    // resolve to first deprecated argument that's set; here, it's the final one
-    conf.set(Property.INSTANCE_DFS_URI, "");
-    assertSame(Property.INSTANCE_DFS_URI, conf.resolve(Property.INSTANCE_VOLUMES,
-        Property.INSTANCE_DFS_DIR, Property.INSTANCE_DFS_URI));
+      // non-deprecated second argument
+      var e2 = assertThrows(IllegalArgumentException.class,
+          () -> conf.resolve(Property.INSTANCE_VOLUMES, Property.INSTANCE_ZK_HOST,
+              Property.INSTANCE_SECRET, Property.INSTANCE_ZK_TIMEOUT, Property.INSTANCE_VOLUMES));
+      assertEquals("Unexpected non-deprecated [INSTANCE_SECRET, INSTANCE_VOLUMES]",
+          e2.getMessage());
 
-    // resolve to first deprecated argument that's set; now, it's the first one because both are set
-    conf.set(Property.INSTANCE_DFS_DIR, "");
-    assertSame(Property.INSTANCE_DFS_DIR, conf.resolve(Property.INSTANCE_VOLUMES,
-        Property.INSTANCE_DFS_DIR, Property.INSTANCE_DFS_URI));
+      // empty second argument always resolves to non-deprecated first argument
+      assertSame(Property.INSTANCE_VOLUMES, conf.resolve(Property.INSTANCE_VOLUMES));
 
-    // every property is set, so resolve to the non-deprecated one
-    conf.set(Property.INSTANCE_VOLUMES, "");
-    assertSame(Property.INSTANCE_VOLUMES, conf.resolve(Property.INSTANCE_VOLUMES,
-        Property.INSTANCE_DFS_DIR, Property.INSTANCE_DFS_URI));
+      // none are set, resolve to non-deprecated
+      assertSame(Property.INSTANCE_VOLUMES, conf.resolve(Property.INSTANCE_VOLUMES,
+          Property.INSTANCE_ZK_HOST, Property.INSTANCE_ZK_TIMEOUT));
+
+      // resolve to first deprecated argument that's set; here, it's the final one
+      conf.set(Property.INSTANCE_ZK_TIMEOUT, "");
+      assertSame(Property.INSTANCE_ZK_TIMEOUT, conf.resolve(Property.INSTANCE_VOLUMES,
+          Property.INSTANCE_ZK_HOST, Property.INSTANCE_ZK_TIMEOUT));
+
+      // resolve to first deprecated argument that's set; now, it's the first one because both are
+      // set
+      conf.set(Property.INSTANCE_ZK_HOST, "");
+      assertSame(Property.INSTANCE_ZK_HOST, conf.resolve(Property.INSTANCE_VOLUMES,
+          Property.INSTANCE_ZK_HOST, Property.INSTANCE_ZK_TIMEOUT));
+
+      // every property is set, so resolve to the non-deprecated one
+      conf.set(Property.INSTANCE_VOLUMES, "");
+      assertSame(Property.INSTANCE_VOLUMES, conf.resolve(Property.INSTANCE_VOLUMES,
+          Property.INSTANCE_ZK_HOST, Property.INSTANCE_ZK_TIMEOUT));
+    } finally {
+      // Reset back to original setting
+      isDeprecatedField.set(Property.INSTANCE_ZK_HOST, origIsDepInstanceZkHost);
+      isDeprecatedField.set(Property.INSTANCE_ZK_TIMEOUT, origIsDepInstanceZkTimeout);
+    }
   }
+
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/ConfigCheckUtilTest.java b/core/src/test/java/org/apache/accumulo/core/conf/ConfigCheckUtilTest.java
index e204316..6741407 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/ConfigCheckUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/ConfigCheckUtilTest.java
@@ -39,7 +39,7 @@
     m.put(Property.MANAGER_CLIENTPORT.getKey(), "9999");
     m.put(Property.MANAGER_TABLET_BALANCER.getKey(),
         "org.apache.accumulo.server.manager.balancer.TableLoadBalancer");
-    m.put(Property.MANAGER_BULK_RETRIES.getKey(), "3");
+    m.put(Property.MANAGER_BULK_TIMEOUT.getKey(), "5m");
     ConfigCheckUtil.validate(m.entrySet(), "test");
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java b/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java
index 1eeae56..67014c5 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java
@@ -20,12 +20,9 @@
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertSame;
-import static org.junit.jupiter.api.Assertions.assertThrows;
 
-import java.util.Arrays;
 import java.util.function.BiConsumer;
 
-import org.apache.commons.configuration2.BaseConfiguration;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -67,34 +64,4 @@
     assertEquals("new.test", newProp2);
   }
 
-  @Test
-  public void testMasterManagerPropertyRename() {
-    Arrays.stream(Property.values()).filter(p -> p.getType() != PropertyType.PREFIX)
-        .filter(p -> p.getKey().startsWith(Property.MANAGER_PREFIX.getKey())).forEach(p -> {
-          String oldProp =
-              "master." + p.getKey().substring(Property.MANAGER_PREFIX.getKey().length());
-          assertEquals(p.getKey(), DeprecatedPropertyUtil.getReplacementName(oldProp, NOOP));
-        });
-  }
-
-  @Test
-  public void testSanityCheckManagerProperties() {
-    var config = new BaseConfiguration();
-    config.setProperty("regular.prop1", "value");
-    config.setProperty("regular.prop2", "value");
-    assertEquals(2, config.size());
-    DeprecatedPropertyUtil.sanityCheckManagerProperties(config); // should succeed
-    config.setProperty("master.deprecatedProp", "value");
-    assertEquals(3, config.size());
-    DeprecatedPropertyUtil.sanityCheckManagerProperties(config); // should succeed
-    config.setProperty("manager.replacementProp", "value");
-    assertEquals(4, config.size());
-    assertThrows(IllegalStateException.class,
-        () -> DeprecatedPropertyUtil.sanityCheckManagerProperties(config),
-        "Sanity check should fail when 'master.*' and 'manager.*' appear in same config");
-    config.clearProperty("master.deprecatedProp");
-    assertEquals(3, config.size());
-    DeprecatedPropertyUtil.sanityCheckManagerProperties(config); // should succeed
-  }
-
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
index b6d38bc..3821214 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
@@ -206,20 +206,19 @@
   // This test verifies all "sensitive" properties are properly marked as sensitive
   @Test
   public void testSensitiveKeys() {
-    // add trace token, because it's a sensitive property not in the default configuration
+    // add instance.crypto.opts, because it's a sensitive property not in the default configuration
     ConfigurationCopy conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    conf.set("trace.token.property.blah", "something");
+    conf.set("instance.crypto.opts.sensitive.blah", "something");
 
     // ignores duplicates because ConfigurationCopy already de-duplicates
     Collector<Entry<String,String>,?,TreeMap<String,String>> treeMapCollector =
         Collectors.toMap(Entry::getKey, Entry::getValue, (a, b) -> a, TreeMap::new);
 
-    @SuppressWarnings("deprecation")
     Predicate<Entry<String,String>> sensitiveNames =
         e -> e.getKey().equals(Property.INSTANCE_SECRET.getKey())
             || e.getKey().toLowerCase().contains("password")
             || e.getKey().toLowerCase().endsWith("secret")
-            || e.getKey().startsWith(Property.TRACE_TOKEN_PROPERTY_PREFIX.getKey());
+            || e.getKey().startsWith(Property.INSTANCE_CRYPTO_SENSITIVE_PREFIX.getKey());
 
     Predicate<Entry<String,String>> isMarkedSensitive = e -> Property.isSensitive(e.getKey());
 
@@ -228,8 +227,8 @@
     TreeMap<String,String> actual = StreamSupport.stream(conf.spliterator(), false)
         .filter(isMarkedSensitive).collect(treeMapCollector);
 
-    // make sure trace token property wasn't excluded from both
-    assertEquals("something", expected.get("trace.token.property.blah"));
+    // make sure instance.crypto.opts property wasn't excluded from both
+    assertEquals("something", expected.get("instance.crypto.opts.sensitive.blah"));
     assertEquals(expected, actual);
   }
 
@@ -251,9 +250,6 @@
     assertTrue(Property.INSTANCE_SECRET.isSensitive());
     assertFalse(Property.INSTANCE_VOLUMES.isSensitive());
 
-    @SuppressWarnings("deprecation")
-    Property deprecatedProp = Property.GENERAL_CLASSPATHS;
-    assertTrue(deprecatedProp.isDeprecated());
     assertFalse(Property.INSTANCE_VOLUMES_REPLACEMENTS.isDeprecated());
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraintTest.java b/core/src/test/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraintTest.java
deleted file mode 100644
index 1e094c4..0000000
--- a/core/src/test/java/org/apache/accumulo/core/constraints/DefaultKeySizeConstraintTest.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.constraints;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Collections;
-
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("deprecation")
-public class DefaultKeySizeConstraintTest {
-
-  Constraint constraint = new DefaultKeySizeConstraint();
-
-  byte[] oversized = new byte[1048577];
-  byte[] large = new byte[419430];
-
-  @Test
-  public void testConstraint() {
-
-    // pass constraints
-    Mutation m = new Mutation("rowId");
-    m.put("colf", "colq", new Value());
-    assertEquals(Collections.emptyList(), constraint.check(null, m));
-
-    // test with row id > 1mb
-    m = new Mutation(oversized);
-    m.put("colf", "colq", new Value());
-    assertEquals(
-        Collections.singletonList(DefaultKeySizeConstraint.MAX__KEY_SIZE_EXCEEDED_VIOLATION),
-        constraint.check(null, m));
-
-    // test with colf > 1mb
-    m = new Mutation("rowid");
-    m.put(new Text(oversized), new Text("colq"), new Value());
-    assertEquals(
-        Collections.singletonList(DefaultKeySizeConstraint.MAX__KEY_SIZE_EXCEEDED_VIOLATION),
-        constraint.check(null, m));
-
-    // test with colf > 1mb
-    m = new Mutation("rowid");
-    m.put(new Text(oversized), new Text("colq"), new Value());
-    assertEquals(
-        Collections.singletonList(DefaultKeySizeConstraint.MAX__KEY_SIZE_EXCEEDED_VIOLATION),
-        constraint.check(null, m));
-
-    // test sum of smaller sizes violates 1mb constraint
-    m = new Mutation(large);
-    m.put(new Text(large), new Text(large), new Value());
-    assertEquals(
-        Collections.singletonList(DefaultKeySizeConstraint.MAX__KEY_SIZE_EXCEEDED_VIOLATION),
-        constraint.check(null, m));
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/constraints/DeprecatedConstraintExtendTest.java b/core/src/test/java/org/apache/accumulo/core/constraints/DeprecatedConstraintExtendTest.java
deleted file mode 100644
index e75bd16..0000000
--- a/core/src/test/java/org/apache/accumulo/core/constraints/DeprecatedConstraintExtendTest.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.constraints;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This tests the case where a user extended a Constraint class before it was deprecated to make
- * sure the old Constraint will still work with the API migration changes.
- *
- * @since 2.1.0
- */
-@SuppressWarnings("deprecation")
-public class DeprecatedConstraintExtendTest {
-
-  byte[] min = new byte[1024];
-  byte[] oversized = new byte[1048577];
-
-  @Test
-  public void testMinKeySizeConstraint() {
-    Constraint constraint = new MinKeySizeConstraint();
-
-    // pass constraints
-    Mutation m = new Mutation(min);
-    m.put("colf", "colq", new Value());
-    assertEquals(Collections.emptyList(), constraint.check(null, m));
-
-    // test with row id < 1KB
-    m = new Mutation("rowid");
-    m.put("colf", "colq", new Value());
-    assertEquals(Collections.singletonList(MinKeySizeConstraint.MIN_KEY_SIZE_EXCEEDED_VIOLATION),
-        constraint.check(null, m));
-
-    // test with colf > 1mb
-    m = new Mutation("rowid");
-    m.put(new Text(oversized), new Text("colq"), new Value());
-    assertEquals(
-        Collections.singletonList(DefaultKeySizeConstraint.MAX__KEY_SIZE_EXCEEDED_VIOLATION),
-        constraint.check(null, m));
-  }
-
-  @Test
-  public void testFoo() {
-    FooConstraint fc = new FooConstraint();
-    // pass constraints
-    Mutation m = new Mutation("blah");
-    m.put("colf", "colq", new Value());
-    assertEquals(null, fc.check(null, m));
-
-    // test fail constraint
-    m = new Mutation("foo");
-    m.put("colf", "colq", new Value());
-    assertEquals(Collections.singletonList(Short.valueOf("1")), fc.check(null, m));
-  }
-
-  /**
-   * Limit the size of 1mb but also a minimum of 1KB
-   */
-  @SuppressFBWarnings(value = "NM_WRONG_PACKAGE",
-      justification = "Same name used for compatibility during deprecation cycle")
-  private static class MinKeySizeConstraint extends DefaultKeySizeConstraint {
-    protected static final short MIN_KEY_SIZE_EXCEEDED_VIOLATION = 2;
-    protected static final long minSize = 1024; // 1MB default size
-
-    @Override
-    public List<Short> check(Constraint.Environment env, Mutation mutation) {
-      List<Short> violations = super.check(env, mutation);
-      if (!violations.isEmpty()) {
-        return violations;
-      }
-
-      for (ColumnUpdate cu : mutation.getUpdates()) {
-        int size = mutation.getRow().length;
-        size += cu.getColumnFamily().length;
-        size += cu.getColumnQualifier().length;
-        size += cu.getColumnVisibility().length;
-
-        if (size < minSize) {
-          violations.add(MIN_KEY_SIZE_EXCEEDED_VIOLATION);
-        }
-      }
-      return violations;
-    }
-  }
-
-  /**
-   * Test previously defined constraint.
-   */
-  public class FooConstraint implements Constraint {
-    public String getViolationDescription(short violationCode) {
-      switch (violationCode) {
-        case 1:
-          return "Contains foo";
-      }
-      throw new IllegalArgumentException();
-    }
-
-    public List<Short> check(Constraint.Environment env, Mutation mutation) {
-      if (new String(mutation.getRow()).contains("foo")) {
-        return Collections.singletonList(Short.valueOf("1"));
-      }
-      return null;
-    }
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/constraints/NoDeleteConstraintTest.java b/core/src/test/java/org/apache/accumulo/core/constraints/NoDeleteConstraintTest.java
deleted file mode 100644
index b2bee5e..0000000
--- a/core/src/test/java/org/apache/accumulo/core/constraints/NoDeleteConstraintTest.java
+++ /dev/null
@@ -1,50 +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 org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.util.List;
-
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("deprecation")
-public class NoDeleteConstraintTest {
-
-  @Test
-  public void testConstraint() {
-    Mutation m1 = new Mutation("r1");
-    m1.putDelete("f1", "q1");
-
-    NoDeleteConstraint ndc = new NoDeleteConstraint();
-
-    List<Short> results = ndc.check(null, m1);
-    assertEquals(1, results.size());
-    assertEquals(1, results.get(0).intValue());
-
-    Mutation m2 = new Mutation("r1");
-    m2.put("f1", "q1", new Value("v1"));
-
-    results = ndc.check(null, m2);
-    assertNull(results);
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/constraints/VisibilityConstraintTest.java b/core/src/test/java/org/apache/accumulo/core/constraints/VisibilityConstraintTest.java
deleted file mode 100644
index e70b402..0000000
--- a/core/src/test/java/org/apache/accumulo/core/constraints/VisibilityConstraintTest.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.constraints;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.createNiceMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.AuthorizationContainer;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("deprecation")
-public class VisibilityConstraintTest {
-
-  VisibilityConstraint vc;
-  Constraint.Environment env;
-  Mutation mutation;
-
-  static final ColumnVisibility good = new ColumnVisibility("good");
-  static final ColumnVisibility bad = new ColumnVisibility("bad");
-
-  static final String D = "don't care";
-
-  static final List<Short> ENOAUTH = Arrays.asList((short) 2);
-
-  @BeforeEach
-  public void setUp() {
-    vc = new VisibilityConstraint();
-    mutation = new Mutation("r");
-
-    ArrayByteSequence bs = new ArrayByteSequence("good".getBytes(UTF_8));
-
-    AuthorizationContainer ac = createNiceMock(AuthorizationContainer.class);
-    expect(ac.contains(bs)).andReturn(true);
-    replay(ac);
-
-    env = createMock(Constraint.Environment.class);
-    expect(env.getAuthorizationsContainer()).andReturn(ac);
-    replay(env);
-  }
-
-  @Test
-  public void testNoVisibility() {
-    mutation.put(D, D, D);
-    assertNull(vc.check(env, mutation), "authorized");
-  }
-
-  @Test
-  public void testVisibilityNoAuth() {
-    mutation.put(D, D, bad, D);
-    assertEquals(ENOAUTH, vc.check(env, mutation), "unauthorized");
-  }
-
-  @Test
-  public void testGoodVisibilityAuth() {
-    mutation.put(D, D, good, D);
-    assertNull(vc.check(env, mutation), "authorized");
-  }
-
-  @Test
-  public void testCachedVisibilities() {
-    mutation.put(D, D, good, "v");
-    mutation.put(D, D, good, "v2");
-    assertNull(vc.check(env, mutation), "authorized");
-  }
-
-  @Test
-  public void testMixedVisibilities() {
-    mutation.put(D, D, bad, D);
-    mutation.put(D, D, good, D);
-    assertEquals(ENOAUTH, vc.check(env, mutation), "unauthorized");
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java b/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
index 4f919aa..e9f5c03 100644
--- a/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/crypto/CryptoTest.java
@@ -23,6 +23,7 @@
 import static org.apache.accumulo.core.crypto.CryptoUtils.getFileDecrypter;
 import static org.apache.accumulo.core.spi.crypto.CryptoEnvironment.Scope.TABLE;
 import static org.apache.accumulo.core.spi.crypto.CryptoEnvironment.Scope.WAL;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
@@ -87,7 +88,6 @@
 
 public class CryptoTest {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final int MARKER_INT = 0xCADEFEDD;
   private static final String MARKER_STRING = "1 2 3 4 5 6 7 8 a b c d e f g h ";
   private static final Configuration hadoopConf = new Configuration();
@@ -409,11 +409,11 @@
       NoSuchProviderException, NoSuchPaddingException, InvalidKeyException {
     // verify valid key sizes (corresponds to 128, 192, and 256 bits)
     for (int i : new int[] {16, 24, 32}) {
-      verifyKeySizeForCBC(random, i);
+      verifyKeySizeForCBC(RANDOM.get(), i);
     }
     // verify invalid key sizes
     for (int i : new int[] {1, 2, 8, 11, 15, 64, 128}) {
-      assertThrows(InvalidKeyException.class, () -> verifyKeySizeForCBC(random, i));
+      assertThrows(InvalidKeyException.class, () -> verifyKeySizeForCBC(RANDOM.get(), i));
     }
   }
 
@@ -429,8 +429,8 @@
   @Test
   public void testAESKeyUtilsWrapAndUnwrap()
       throws NoSuchAlgorithmException, NoSuchProviderException {
-    java.security.Key kek = AESCryptoService.generateKey(random, 16);
-    java.security.Key fek = AESCryptoService.generateKey(random, 16);
+    java.security.Key kek = AESCryptoService.generateKey(RANDOM.get(), 16);
+    java.security.Key fek = AESCryptoService.generateKey(RANDOM.get(), 16);
     byte[] wrapped = AESCryptoService.wrapKey(fek, kek);
     assertFalse(Arrays.equals(fek.getEncoded(), wrapped));
     java.security.Key unwrapped = AESCryptoService.unwrapKey(wrapped, kek);
@@ -440,8 +440,8 @@
   @Test
   public void testAESKeyUtilsFailUnwrapWithWrongKEK()
       throws NoSuchAlgorithmException, NoSuchProviderException {
-    java.security.Key kek = AESCryptoService.generateKey(random, 16);
-    java.security.Key fek = AESCryptoService.generateKey(random, 16);
+    java.security.Key kek = AESCryptoService.generateKey(RANDOM.get(), 16);
+    java.security.Key fek = AESCryptoService.generateKey(RANDOM.get(), 16);
     byte[] wrongBytes = kek.getEncoded();
     wrongBytes[0]++;
     java.security.Key wrongKek = new SecretKeySpec(wrongBytes, "AES");
diff --git a/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java b/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java
index f7ce7cf..acb3eb9 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/TableIdTest.java
@@ -25,8 +25,7 @@
 import static org.junit.jupiter.api.Assertions.fail;
 
 import org.apache.accumulo.core.WithTestNames;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
 import org.slf4j.Logger;
@@ -50,13 +49,9 @@
   @Test
   public void testCacheNoDuplicates() {
 
-    @SuppressWarnings("deprecation")
-    TableId REPL_TABLE_ID = org.apache.accumulo.core.replication.ReplicationTable.ID;
-
-    // the next two lines just preloads the built-ins, since they now exist in a separate class from
+    // the next line just preloads the built-ins, since they now exist in a separate class from
     // TableId, and aren't preloaded when the TableId class is referenced
-    assertNotSame(RootTable.ID, MetadataTable.ID);
-    assertNotSame(RootTable.ID, REPL_TABLE_ID);
+    assertNotSame(AccumuloTable.ROOT.tableId(), AccumuloTable.METADATA.tableId());
 
     String tableString = "table-" + testName();
     long initialSize = cacheCount();
@@ -66,11 +61,9 @@
 
     // ensure duplicates are not created
     TableId builtInTableId = TableId.of("!0");
-    assertSame(MetadataTable.ID, builtInTableId);
+    assertSame(AccumuloTable.METADATA.tableId(), builtInTableId);
     builtInTableId = TableId.of("+r");
-    assertSame(RootTable.ID, builtInTableId);
-    builtInTableId = TableId.of("+rep");
-    assertSame(REPL_TABLE_ID, builtInTableId);
+    assertSame(AccumuloTable.ROOT.tableId(), builtInTableId);
     table1 = TableId.of(tableString);
     assertEquals(initialSize + 1, cacheCount());
     assertEquals(tableString, table1.canonical());
@@ -98,7 +91,7 @@
     long preGCSize = 0;
     int i = 0;
     while ((preGCSize = cacheCount()) < 100) {
-      TableId.of(new String("table" + i++));
+      TableId.of(("table" + i++));
     }
     LOG.info("Entries before System.gc(): {}", preGCSize);
     assertEquals(100, preGCSize);
diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.java b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.java
new file mode 100644
index 0000000..4f9315b
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooUtilTest.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.fate.zookeeper;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ZooUtilTest {
+  Logger log = LoggerFactory.getLogger(ZooUtilTest.class);
+
+  @Test
+  void checkUnmodifiable() throws Exception {
+    assertTrue(validateACL(ZooUtil.PRIVATE));
+    assertTrue(validateACL(ZooUtil.PUBLIC));
+  }
+
+  @Test
+  public void checkImmutableAcl() throws Exception {
+
+    final List<ACL> mutable = new ArrayList<>(ZooDefs.Ids.CREATOR_ALL_ACL);
+    assertTrue(validateACL(mutable));
+
+    // Replicates the acl check in ZooKeeper.java to show ZooKeeper will not accept an
+    // ImmutableCollection for the ACL list. ZooKeeper (as of 3.8.1) calls
+    // acl.contains((Object) null) which throws a NPE when passed an immutable collectionCallers
+    // because the way ImmutableCollections.contains() handles nulls (JDK-8265905)
+    try {
+      final List<ACL> immutable = List.copyOf(ZooDefs.Ids.CREATOR_ALL_ACL);
+      assertThrows(NullPointerException.class, () -> validateACL(immutable));
+    } catch (Exception ex) {
+      log.warn("validateAcls failed with exception", ex);
+    }
+  }
+
+  // Copied from ZooKeeper 3.8.1 for stand-alone testing here
+  // https://github.com/apache/zookeeper/blob/2e9c3f3ceda90aeb9380acc87b253bf7661b7794/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java#L3075/
+  private boolean validateACL(List<ACL> acl) throws KeeperException.InvalidACLException {
+    if (acl == null || acl.isEmpty() || acl.contains((Object) null)) {
+      throw new KeeperException.InvalidACLException();
+    }
+    return true;
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java b/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java
index 1d0b5ef..3857731 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/BloomFilterLayerLookupTest.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.core.file;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -37,9 +37,11 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.keyfunctor.ColumnFamilyFunctor;
 import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 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.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
@@ -52,7 +54,6 @@
 public class BloomFilterLayerLookupTest extends WithTestNames {
 
   private static final Logger log = LoggerFactory.getLogger(BloomFilterLayerLookupTest.class);
-  private static final SecureRandom random = new SecureRandom();
 
   @TempDir
   private static File tempDir;
@@ -61,7 +62,7 @@
   public void test() throws IOException {
     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);
@@ -81,8 +82,9 @@
     String suffix = FileOperations.getNewFileExtension(acuconf);
     String fname = new File(tempDir, testName() + "." + suffix).getAbsolutePath();
     FileSKVWriter bmfw = FileOperations.getInstance().newWriterBuilder()
-        .forFile(fname, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(acuconf)
-        .build();
+        .forFile(UnreferencedTabletFile.of(fs, new Path(fname)), fs, conf,
+            NoCryptoServiceFactory.NONE)
+        .withTableConfiguration(acuconf).build();
 
     // write data to file
     long t1 = System.currentTimeMillis();
@@ -99,15 +101,16 @@
 
     t1 = System.currentTimeMillis();
     FileSKVIterator bmfr = FileOperations.getInstance().newReaderBuilder()
-        .forFile(fname, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(acuconf)
-        .build();
+        .forFile(UnreferencedTabletFile.of(fs, new Path(fname)), fs, conf,
+            NoCryptoServiceFactory.NONE)
+        .withTableConfiguration(acuconf).build();
     t2 = System.currentTimeMillis();
     log.debug("Opened {} in {}", fname, (t2 - t1));
 
     int hits = 0;
     t1 = System.currentTimeMillis();
     for (int i = 0; i < 5000; i++) {
-      int row = random.nextInt(Integer.MAX_VALUE);
+      int row = RANDOM.get().nextInt(Integer.MAX_VALUE);
       seek(bmfr, row);
       if (valsSet.contains(row)) {
         hits++;
diff --git a/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java b/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java
index b4df990..b85004a 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/FileOperationsTest.java
@@ -26,6 +26,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -52,9 +53,9 @@
       Configuration conf = new Configuration();
       FileSystem fs = FileSystem.getLocal(conf);
       AccumuloConfiguration acuconf = DefaultConfiguration.getInstance();
-      writer =
-          fileOperations.newWriterBuilder().forFile(filename, fs, conf, NoCryptoServiceFactory.NONE)
-              .withTableConfiguration(acuconf).build();
+      writer = fileOperations.newWriterBuilder()
+          .forFile(UnreferencedTabletFile.of(fs, testFile), fs, conf, NoCryptoServiceFactory.NONE)
+          .withTableConfiguration(acuconf).build();
       writer.close();
     } catch (Exception ex) {
       caughtException = true;
diff --git a/core/src/test/java/org/apache/accumulo/core/file/FilePrefixTest.java b/core/src/test/java/org/apache/accumulo/core/file/FilePrefixTest.java
new file mode 100644
index 0000000..7b84fa9
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/file/FilePrefixTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import org.junit.jupiter.api.Test;
+
+public class FilePrefixTest {
+
+  @Test
+  public void testPrefixes() {
+    assertEquals(FilePrefix.BULK_IMPORT, FilePrefix.fromPrefix("I"));
+    assertEquals(FilePrefix.MINOR_COMPACTION, FilePrefix.fromPrefix("F"));
+    assertEquals(FilePrefix.MAJOR_COMPACTION, FilePrefix.fromPrefix("C"));
+    assertEquals(FilePrefix.MAJOR_COMPACTION_ALL_FILES, FilePrefix.fromPrefix("A"));
+    assertThrows(IllegalArgumentException.class, () -> {
+      FilePrefix.fromPrefix("B");
+    });
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/BlockCacheManagerTest.java b/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/BlockCacheManagerTest.java
deleted file mode 100644
index e8b20a8..0000000
--- a/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/BlockCacheManagerTest.java
+++ /dev/null
@@ -1,42 +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.blockfile.cache;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import org.apache.accumulo.core.spi.cache.BlockCacheManager;
-import org.apache.accumulo.core.spi.cache.CacheType;
-import org.junit.jupiter.api.Test;
-
-public class BlockCacheManagerTest {
-
-  @Test
-  @SuppressWarnings("deprecation")
-  public void testGetPropertyPrefix() {
-    assertEquals("tserver.cache.config.lru.data.",
-        BlockCacheManager.getFullyQualifiedPropertyPrefix("lru", CacheType.DATA));
-    assertEquals("tserver.cache.config.lru.index.",
-        BlockCacheManager.getFullyQualifiedPropertyPrefix("lru", CacheType.INDEX));
-    assertEquals("tserver.cache.config.lru.summary.",
-        BlockCacheManager.getFullyQualifiedPropertyPrefix("lru", CacheType.SUMMARY));
-    assertEquals("tserver.cache.config.lru.default.",
-        BlockCacheManager.getFullyQualifiedPropertyPrefix("lru"));
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java b/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java
index b6b0ea3..bf814e0 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.file.blockfile.cache;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -25,8 +26,6 @@
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.security.SecureRandom;
-
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -52,8 +51,6 @@
  */
 public class TestLruBlockCache {
 
-  private static final SecureRandom random = new SecureRandom();
-
   @Test
   public void testConfiguration() {
     ConfigurationCopy cc = new ConfigurationCopy();
@@ -506,7 +503,7 @@
   private Block[] generateRandomBlocks(int numBlocks, long maxSize) {
     Block[] blocks = new Block[numBlocks];
     for (int i = 0; i < numBlocks; i++) {
-      blocks[i] = new Block("block" + i, random.nextInt((int) maxSize) + 1);
+      blocks[i] = new Block("block" + i, RANDOM.get().nextInt((int) maxSize) + 1);
     }
     return blocks;
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java
new file mode 100644
index 0000000..a220407
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java
@@ -0,0 +1,264 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.security.SecureRandom;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.accumulo.core.client.sample.Sampler;
+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.crypto.CryptoFactoryLoader;
+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.file.FileSKVIterator;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheManagerFactory;
+import org.apache.accumulo.core.file.blockfile.cache.lru.LruBlockCache;
+import org.apache.accumulo.core.file.blockfile.cache.lru.LruBlockCacheManager;
+import org.apache.accumulo.core.file.blockfile.impl.BasicCacheProvider;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
+import org.apache.accumulo.core.file.rfile.RFile.FencedReader;
+import org.apache.accumulo.core.file.rfile.RFile.Reader;
+import org.apache.accumulo.core.file.rfile.RFileTest.SeekableByteArrayInputStream;
+import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iteratorsImpl.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.accumulo.core.sample.impl.SamplerFactory;
+import org.apache.accumulo.core.spi.cache.BlockCacheManager;
+import org.apache.accumulo.core.spi.cache.CacheType;
+import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
+import org.apache.accumulo.core.spi.crypto.CryptoService;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+
+public abstract class AbstractRFileTest {
+
+  protected static final SecureRandom random = new SecureRandom();
+  protected static final Collection<ByteSequence> EMPTY_COL_FAMS = List.of();
+
+  protected AccumuloConfiguration conf = null;
+
+  public static class TestRFile {
+
+    protected Configuration conf = new Configuration();
+    public RFile.Writer writer;
+    protected ByteArrayOutputStream baos;
+    protected FSDataOutputStream dos;
+    protected SeekableByteArrayInputStream bais;
+    protected FSDataInputStream in;
+    protected AccumuloConfiguration accumuloConfiguration;
+    public Reader reader;
+    public SortedKeyValueIterator<Key,Value> iter;
+    private BlockCacheManager manager;
+
+    public TestRFile(AccumuloConfiguration accumuloConfiguration) {
+      this.accumuloConfiguration = accumuloConfiguration;
+      if (this.accumuloConfiguration == null) {
+        this.accumuloConfiguration = DefaultConfiguration.getInstance();
+      }
+    }
+
+    public void openWriter(boolean startDLG) throws IOException {
+      openWriter(startDLG, 1000);
+    }
+
+    public void openWriter(boolean startDLG, int blockSize) throws IOException {
+      baos = new ByteArrayOutputStream();
+      dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
+      CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE,
+          accumuloConfiguration.getAllCryptoProperties());
+
+      BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", conf, cs);
+
+      SamplerConfigurationImpl samplerConfig =
+          SamplerConfigurationImpl.newSamplerConfig(accumuloConfiguration);
+      Sampler sampler = null;
+
+      if (samplerConfig != null) {
+        sampler = SamplerFactory.newSampler(samplerConfig, accumuloConfiguration);
+      }
+
+      writer = new RFile.Writer(_cbw, blockSize, 1000, samplerConfig, sampler);
+
+      if (startDLG) {
+        writer.startDefaultLocalityGroup();
+      }
+    }
+
+    public void openWriter() throws IOException {
+      openWriter(1000);
+    }
+
+    public void openWriter(int blockSize) throws IOException {
+      openWriter(true, blockSize);
+    }
+
+    public void closeWriter() throws IOException {
+      dos.flush();
+      writer.close();
+      dos.close();
+      if (baos != null) {
+        baos.close();
+      }
+    }
+
+    public void openReader(Range fence) throws IOException {
+      openReader(true, fence);
+    }
+
+    public void openReader() throws IOException {
+      openReader(true);
+    }
+
+    public void openReader(boolean cfsi) throws IOException {
+      openReader(cfsi, null);
+    }
+
+    public void openReader(boolean cfsi, Range fence) throws IOException {
+      int fileLength = 0;
+      byte[] data = null;
+      data = baos.toByteArray();
+
+      bais = new SeekableByteArrayInputStream(data);
+      in = new FSDataInputStream(bais);
+      fileLength = data.length;
+
+      DefaultConfiguration dc = DefaultConfiguration.getInstance();
+      ConfigurationCopy cc = new ConfigurationCopy(dc);
+      cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
+      try {
+        manager = BlockCacheManagerFactory.getInstance(cc);
+      } catch (ReflectiveOperationException e) {
+        throw new IllegalStateException("Error creating BlockCacheManager", e);
+      }
+      cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(100000));
+      cc.set(Property.TSERV_DATACACHE_SIZE, Long.toString(100000000));
+      cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(100000000));
+      manager.start(BlockCacheConfiguration.forTabletServer(cc));
+      LruBlockCache indexCache = (LruBlockCache) manager.getBlockCache(CacheType.INDEX);
+      LruBlockCache dataCache = (LruBlockCache) manager.getBlockCache(CacheType.DATA);
+
+      CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE,
+          accumuloConfiguration.getAllCryptoProperties());
+
+      CachableBuilder cb = new CachableBuilder().input(in, "source-1").length(fileLength).conf(conf)
+          .cacheProvider(new BasicCacheProvider(indexCache, dataCache)).cryptoService(cs);
+      reader = new RFile.Reader(cb);
+      if (cfsi) {
+        iter = new ColumnFamilySkippingIterator(reader);
+      }
+      if (fence != null) {
+        iter = new FencedReader(reader, fence);
+      }
+
+      checkIndex(reader);
+    }
+
+    public void closeReader() throws IOException {
+      reader.close();
+      in.close();
+      if (null != manager) {
+        manager.stop();
+      }
+    }
+
+    public void seek(Key nk) throws IOException {
+      iter.seek(new Range(nk, null), EMPTY_COL_FAMS, false);
+    }
+  }
+
+  protected static void checkIndex(Reader reader) throws IOException {
+    FileSKVIterator indexIter = reader.getIndex();
+
+    if (indexIter.hasTop()) {
+      Key lastKey = new Key(indexIter.getTopKey());
+
+      if (reader.getFirstRow().compareTo(lastKey.getRow()) > 0) {
+        throw new IllegalStateException(
+            "First key out of order " + reader.getFirstRow() + " " + lastKey);
+      }
+
+      indexIter.next();
+
+      while (indexIter.hasTop()) {
+        if (lastKey.compareTo(indexIter.getTopKey()) > 0) {
+          throw new IllegalStateException(
+              "Indext out of order " + lastKey + " " + indexIter.getTopKey());
+        }
+
+        lastKey = new Key(indexIter.getTopKey());
+        indexIter.next();
+
+      }
+
+      if (!reader.getLastRow().equals(lastKey.getRow())) {
+        throw new IllegalStateException(
+            "Last key out of order " + reader.getLastRow() + " " + lastKey);
+      }
+    }
+  }
+
+  static Key newKey(String row, String cf, String cq, String cv, long ts) {
+    return new Key(row.getBytes(), cf.getBytes(), cq.getBytes(), cv.getBytes(), ts);
+  }
+
+  static Value newValue(String val) {
+    return new Value(val);
+  }
+
+  static String formatString(String prefix, int i) {
+    return String.format(prefix + "%06d", i);
+  }
+
+  protected void verify(TestRFile trf, Iterator<Key> eki, Iterator<Value> evi) throws IOException {
+    verify(trf.iter, eki, evi);
+  }
+
+  protected void verify(SortedKeyValueIterator<Key,Value> iter, Iterator<Key> eki,
+      Iterator<Value> evi) throws IOException {
+
+    while (iter.hasTop()) {
+      Key ek = eki.next();
+      Value ev = evi.next();
+
+      assertEquals(ek, iter.getTopKey());
+      assertEquals(ev, iter.getTopValue());
+
+      iter.next();
+    }
+
+    assertFalse(eki.hasNext());
+    assertFalse(evi.hasNext());
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java
new file mode 100644
index 0000000..3698216
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java
@@ -0,0 +1,509 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.crypto.CryptoTest;
+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.file.FileSKVIterator;
+import org.apache.accumulo.core.file.rfile.RFile.FencedIndex;
+import org.apache.accumulo.core.file.rfile.RFile.FencedReader;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException;
+import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
+import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+public class FencedRFileTest extends AbstractRFileTest {
+
+  @BeforeAll
+  public static void setupCryptoKeyFile() throws Exception {
+    CryptoTest.setupKeyFiles(FencedRFileTest.class);
+  }
+
+  @Test
+  public void testFencingNoRange() throws IOException {
+    // Test with infinite start/end range
+    // Expect entire range to be seen
+    assertEquals(1024, testFencing(List.of(new Range()), List.of(new Range())));
+  }
+
+  @Test
+  public void testFencing1() throws IOException {
+    // Test with fenced starting range at beginning and infinite end range
+    // Expect entire range to be seen
+    assertEquals(1024, testFencing(List.of(new Range("r_000000", null)), List.of(new Range())));
+  }
+
+  @Test
+  public void testFencing2() throws IOException {
+    // Test with 2 ranges that are continuous which should be merged
+    // Expect entire all rows to be seen as first range end key is inclusive
+    assertEquals(1024,
+        testFencing(List.of(new Range(null, new Key("r_000002")), new Range("r_000002", null)),
+            List.of(new Range())));
+  }
+
+  @Test
+  public void testFencing3() throws IOException {
+    // Create a fence that contains only row 0 row 2
+    // Expect only to see values from those two rows and not row 1 or row 3
+    final List<Range> ranges = List.of(new Range("r_000000"), new Range("r_000002"));
+
+    // Use the same range for the fence and testing to make sure only the expected keys were seen
+    // Should only be 512 keys as 2 rows * 256
+    assertEquals(512, testFencing(ranges, ranges));
+  }
+
+  // Should fail
+  @Test
+  public void testFencing4() throws IOException {
+    // Create a fence that contains row 0 and row 2 only
+    final List<Range> ranges = List.of(new Range("r_000000"), new Range("r_000002"));
+
+    // Expected range contains only row 2 so should fail as row 1 should also be seen
+    final List<Range> ranges2 = List.of(new Range("r_000002"));
+
+    boolean failed = false;
+    try {
+      testFencing(ranges, ranges2);
+    } catch (AssertionError e) {
+      // expected
+      failed = true;
+    }
+
+    assertTrue(failed, "should have failed");
+  }
+
+  @Test
+  public void testFencing5() throws IOException {
+    // Test all 4 rows individually, should expect entire file
+    final List<Range> ranges = List.of(new Range("r_000000"), new Range("r_000001"),
+        new Range("r_000002"), new Range("r_000003"));
+
+    assertEquals(1024, testFencing(ranges, List.of(new Range())));
+  }
+
+  @Test
+  public void testFencing6() throws IOException {
+    // Set range to 2.5 rows out of 4
+    // Skip row 0, start row 1 and CF 2 (middle row 1), include row 3/4)
+    Key start = Key.builder().row("r_000001").family("cf_000002").build();
+
+    // Create a fence that starts at partial row 1
+    final List<Range> ranges = List.of(new Range(start, true, null, true));
+
+    // 2.5 rows equals 640 keys as each row contains 256 mutations (1024 total across all 4 rows)
+    assertEquals(640, testFencing(ranges, ranges));
+  }
+
+  @Test
+  public void testFencing7() throws IOException {
+    // Set range to 3/4 of 1 row spanning part of row 1 and row 2
+    Key start = Key.builder().row("r_000001").family("cf_000002").build();
+    Key end = Key.builder().row("r_000002").family("cf_000001").build();
+
+    // Create a fence
+    final List<Range> ranges = List.of(new Range(start, true, end, true));
+
+    // 3/4 of 1 rows equals 192 keys as each row contains 256 mutations
+    assertEquals(192, testFencing(ranges, ranges));
+  }
+
+  @Test
+  public void testFencing8() throws IOException {
+    // Create a fence for 2 rows
+    final List<Range> ranges = List.of(new Range("r_000001", true, "r_000002", true));
+
+    // Should only be rows 1 and 2
+    assertEquals(512, testFencing(ranges, ranges));
+  }
+
+  @Test
+  public void testFencing9() throws IOException {
+    // Test out of order ranges that should still cover whole file.
+    final List<Range> ranges = List.of(new Range("r_000002"), new Range("r_000003"),
+        new Range("r_000000"), new Range("r_000001"));
+
+    assertEquals(1024, testFencing(ranges, List.of(new Range())));
+  }
+
+  @Test
+  public void testFencing10() throws IOException {
+    // Test overlap 2 rows that are merged
+    final List<Range> ranges = Range.mergeOverlapping(
+        List.of(new Range("r_000002"), new Range("r_000002", true, "r_000003", true)));
+
+    assertEquals(512, testFencing(ranges, ranges));
+  }
+
+  @Test
+  public void testFencing11() throws IOException {
+    // Test fence covering just a single row
+    final List<Range> ranges = List.of(new Range("r_000001"));
+
+    // should be 256 keys in row r_000001
+    assertEquals(256, testFencing(ranges, ranges));
+  }
+
+  @Test
+  public void testFencing12() throws IOException {
+    final TestRFile trf = initTestFile();
+
+    // Fence off the file to contain only 1 row (r_00001)
+    Range range = new Range(new Range("r_000001"));
+    trf.openReader(range);
+
+    // Open a fenced reader
+    final SortedKeyValueIterator<Key,Value> iter = trf.iter;
+    assertTrue(iter instanceof FencedReader);
+
+    // Seek to the row that is part of the fence
+    seek(iter, new Key(new Text("r_000001")));
+    assertTrue(iter.hasTop());
+
+    // each row has 256 keys, read 1/4 of the keys
+    // and verify hasTop() is true
+    for (int i = 0; i < 64; i++) {
+      iter.next();
+      assertTrue(iter.hasTop());
+    }
+
+    // Seek to a range that is disjoint. The fence only covers
+    // row r_000001 as end row is exclusive so seeking to row r_000002
+    // should result in hasTop() returning false
+    seek(iter, new Key(new Text("r_000002")));
+    // Verify hasTop() is now false
+    assertFalse(iter.hasTop());
+  }
+
+  @Test
+  public void testFirstAndLastRow() throws IOException {
+    final TestRFile trf = initTestFile();
+
+    Text firstRowInFile = new Text(formatString("r_", 0));
+    Text lastRowInFile = new Text(formatString("r_", 3));
+
+    // Infinite range fence
+    // Should just be first/last rows of file
+    assertReader(trf, new Range(), (reader) -> {
+      assertEquals(firstRowInFile, reader.getFirstRow());
+      assertEquals(lastRowInFile, reader.getLastRow());
+    });
+
+    // Range inside of file so should return the rows of the fence
+    assertReader(trf, new Range("r_000001", "r_000002"), (reader) -> {
+      assertEquals(new Text("r_000001"), reader.getFirstRow());
+      assertEquals(new Text("r_000002"), reader.getLastRow());
+    });
+
+    // Test infinite start row
+    assertReader(trf, new Range(null, "r_000001"), (reader) -> {
+      assertEquals(firstRowInFile, reader.getFirstRow());
+      assertEquals(new Text("r_000001"), reader.getLastRow());
+    });
+
+    // Test infinite end row
+    assertReader(trf, new Range("r_000002", null), (reader) -> {
+      assertEquals(new Text("r_000002"), reader.getFirstRow());
+      assertEquals(lastRowInFile, reader.getLastRow());
+    });
+
+    // Test start row matches start of file
+    assertReader(trf, new Range("r_000000", "r_000002"), (reader) -> {
+      // start row of range matches first row in file so that should be returned instead
+      assertEquals(firstRowInFile, reader.getFirstRow());
+      assertEquals(new Text("r_000002"), reader.getLastRow());
+    });
+
+    // Test end row matches end of file
+    assertReader(trf, new Range("r_000001", "r_000003"), (reader) -> {
+      assertEquals(new Text("r_000001"), reader.getFirstRow());
+      // end row of range matches last row in file so that should be returned instead
+      assertEquals(lastRowInFile, reader.getLastRow());
+    });
+
+    // Test case where rows in range are less than and greater than rows in file
+    assertReader(trf, new Range("a", "z"), (reader) -> {
+      assertEquals(firstRowInFile, reader.getFirstRow());
+      assertEquals(lastRowInFile, reader.getLastRow());
+    });
+
+    // Test inclusive end key, usually a row range is required to be an exclusive key
+    // for a tablet file but the fenced reader still supports any range type
+    assertReader(trf, new Range(new Key("r_000002"), true, new Key("r_000002"), true), (reader) -> {
+      assertEquals(new Text("r_000002"), reader.getFirstRow());
+      assertEquals(new Text("r_000002"), reader.getLastRow());
+    });
+
+  }
+
+  @Test
+  public void testUnsupportedMethods() throws IOException {
+    final TestRFile trf = initTestFile();
+    trf.openReader(new Range());
+    FencedReader reader = (FencedReader) trf.iter;
+    FencedIndex index = (FencedIndex) reader.getIndex();
+
+    assertThrows(UnsupportedOperationException.class, () -> reader.init(null, null, null));
+    assertThrows(UnsupportedOperationException.class,
+        () -> index.getSample(new SamplerConfigurationImpl()));
+    assertThrows(UnsupportedOperationException.class,
+        () -> index.seek(new Range(), List.of(), false));
+    assertThrows(UnsupportedOperationException.class, () -> index.deepCopy(null));
+  }
+
+  @Test
+  public void testSetInterrupted() throws IOException {
+    final TestRFile trf = initTestFile();
+    trf.openReader(new Range());
+    FencedReader reader = (FencedReader) trf.iter;
+
+    reader.setInterruptFlag(new AtomicBoolean(true));
+    assertThrows(IterationInterruptedException.class,
+        () -> reader.seek(new Range("r_000001"), List.of(), false));
+
+  }
+
+  @Test
+  public void testReset() throws IOException {
+    final TestRFile trf = initTestFile();
+    trf.openReader(new Range());
+    FencedReader reader = (FencedReader) trf.iter;
+
+    assertFalse(reader.hasTop());
+    reader.seek(new Range("r_000001"), List.of(), false);
+    assertTrue(reader.hasTop());
+    assertEquals(
+        newKey(formatString("r_", 1), formatString("cf_", 0), formatString("cq_", 0), "A", 4),
+        reader.getTopKey());
+
+    reader.reset();
+    assertFalse(reader.hasTop());
+  }
+
+  private int testFencing(List<Range> fencedRange, List<Range> expectedRange) throws IOException {
+    // test an rfile with multiple rows having multiple columns
+
+    final ArrayList<Key> expectedKeys = new ArrayList<>(10000);
+    final ArrayList<Value> expectedValues = new ArrayList<>(10000);
+
+    final List<TestRFile> rangedTrfs = new ArrayList<>();
+    final List<SortedKeyValueIterator<Key,Value>> rangedIters = new ArrayList<>();
+
+    // For each range build a new test rfile and ranged reader for it
+    // We have to copy the data for each range for the test
+    for (Range range : fencedRange) {
+      expectedKeys.clear();
+      expectedValues.clear();
+      final TestRFile trf = new TestRFile(conf);
+      trf.openWriter();
+      writeTestFile(trf, expectedKeys, expectedValues, expectedRange);
+      trf.closeWriter();
+      rangedTrfs.add(trf);
+      trf.openReader(range);
+      rangedIters.add(trf.iter);
+    }
+
+    final MultiIterator trfIter = new MultiIterator(rangedIters, false);
+
+    // seek before everything
+    trfIter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false);
+    verify(trfIter, expectedKeys.iterator(), expectedValues.iterator());
+
+    // seek to the middle
+    int index = expectedKeys.size() / 2;
+    seek(trfIter, expectedKeys.get(index));
+    verify(trfIter, expectedKeys.subList(index, expectedKeys.size()).iterator(),
+        expectedValues.subList(index, expectedKeys.size()).iterator());
+
+    // seek the first key
+    index = 0;
+    seek(trfIter, expectedKeys.get(index));
+    verify(trfIter, expectedKeys.subList(index, expectedKeys.size()).iterator(),
+        expectedValues.subList(index, expectedKeys.size()).iterator());
+
+    // seek to the last key
+    index = expectedKeys.size() - 1;
+    seek(trfIter, expectedKeys.get(index));
+    verify(trfIter, expectedKeys.subList(index, expectedKeys.size()).iterator(),
+        expectedValues.subList(index, expectedKeys.size()).iterator());
+
+    // seek after everything
+    index = expectedKeys.size();
+    seek(trfIter, new Key(new Text("z")));
+    verify(trfIter, expectedKeys.subList(index, expectedKeys.size()).iterator(),
+        expectedValues.subList(index, expectedKeys.size()).iterator());
+
+    // test seeking to the current location
+    index = expectedKeys.size() / 2;
+    seek(trfIter, expectedKeys.get(index));
+    assertTrue(trfIter.hasTop());
+    assertEquals(expectedKeys.get(index), trfIter.getTopKey());
+    assertEquals(expectedValues.get(index), trfIter.getTopValue());
+
+    trfIter.next();
+    index++;
+    assertTrue(trfIter.hasTop());
+    assertEquals(expectedKeys.get(index), trfIter.getTopKey());
+    assertEquals(expectedValues.get(index), trfIter.getTopValue());
+
+    seek(trfIter, expectedKeys.get(index));
+
+    assertTrue(trfIter.hasTop());
+    assertEquals(expectedKeys.get(index), trfIter.getTopKey());
+    assertEquals(expectedValues.get(index), trfIter.getTopValue());
+
+    // test seeking to each location in the file
+    index = 0;
+    for (Key key : expectedKeys) {
+      seek(trfIter, key);
+      assertTrue(trfIter.hasTop());
+      assertEquals(key, trfIter.getTopKey());
+      assertEquals(expectedValues.get(index), trfIter.getTopValue());
+      index++;
+    }
+
+    // test seeking backwards to each key
+    for (int i = expectedKeys.size() - 1; i >= 0; i--) {
+      Key key = expectedKeys.get(i);
+
+      seek(trfIter, key);
+      assertTrue(trfIter.hasTop());
+      assertEquals(key, trfIter.getTopKey());
+      assertEquals(expectedValues.get(i), trfIter.getTopValue());
+
+      if (i - 1 > 0) {
+        // Key pkey =
+        expectedKeys.get(i - 1);
+        // assertEquals(pkey, trf.reader.getPrevKey());
+      }
+    }
+
+    // test seeking to random location and reading all data from that point
+    // there was an off by one bug with this in the transient index
+    for (int i = 0; i < 12; i++) {
+      index = random.nextInt(expectedKeys.size());
+      seek(trfIter, expectedKeys.get(index));
+      for (; index < expectedKeys.size(); index++) {
+        assertTrue(trfIter.hasTop());
+        assertEquals(expectedKeys.get(index), trfIter.getTopKey());
+        assertEquals(expectedValues.get(index), trfIter.getTopValue());
+        trfIter.next();
+      }
+    }
+
+    for (TestRFile rangedTrf : rangedTrfs) {
+      // check index entries to verify within range
+      FileSKVIterator iiter = ((FencedReader) rangedTrf.iter).getIndex();
+      while (iiter.hasTop()) {
+        assertTrue(expectedRange.stream().anyMatch(range -> range.contains(iiter.getTopKey())));
+        iiter.next();
+      }
+      rangedTrf.closeReader();
+    }
+
+    return expectedKeys.size();
+  }
+
+  private static void seek(SortedKeyValueIterator<Key,Value> iter, Key nk) throws IOException {
+    iter.seek(new Range(nk, null), EMPTY_COL_FAMS, false);
+  }
+
+  private void writeTestFile(final TestRFile trf) throws IOException {
+    writeTestFile(trf, null, null, null);
+  }
+
+  private void writeTestFile(final TestRFile trf, final List<Key> expectedKeys,
+      final List<Value> expectedValues, List<Range> expectedRange) throws IOException {
+    int val = 0;
+
+    for (int row = 0; row < 4; row++) {
+      String rowS = formatString("r_", row);
+      for (int cf = 0; cf < 4; cf++) {
+        String cfS = formatString("cf_", cf);
+        for (int cq = 0; cq < 4; cq++) {
+          String cqS = formatString("cq_", cq);
+          for (int cv = 'A'; cv < 'A' + 4; cv++) {
+            String cvS = "" + (char) cv;
+            for (int ts = 4; ts > 0; ts--) {
+              Key k = newKey(rowS, cfS, cqS, cvS, ts);
+              // check below ensures when all key sizes are same more than one index block is
+              // created
+              assertEquals(27, k.getSize());
+              Value v = newValue("" + val);
+              trf.writer.append(k, v);
+              final Key finalK = k;
+              Optional.ofNullable(expectedRange).ifPresent(expected -> {
+                if (expected.stream().anyMatch(range -> range.contains(finalK))) {
+                  expectedKeys.add(k);
+                  expectedValues.add(v);
+                }
+              });
+              val++;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private TestRFile initTestFile() throws IOException {
+    final TestRFile trf = new TestRFile(conf);
+    trf.openWriter();
+    writeTestFile(trf);
+    trf.closeWriter();
+    return trf;
+  }
+
+  private static void assertReader(final TestRFile trf, Range range,
+      ThrowableConsumer<FencedReader,IOException> run) throws IOException {
+    FencedReader reader = null;
+    try {
+      trf.openReader(range);
+      reader = (FencedReader) trf.iter;
+      run.accept(reader);
+    } finally {
+      if (reader != null) {
+        reader.close();
+      }
+    }
+
+  }
+
+  // Similar to the java.util.function.Consumer interface but throws an exception
+  interface ThrowableConsumer<T,U extends Throwable> {
+    void accept(T t) throws U;
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
index 183db71..2065f8b 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
@@ -18,13 +18,13 @@
  */
 package org.apache.accumulo.core.file.rfile;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.UncheckedIOException;
-import java.security.SecureRandom;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
@@ -47,7 +47,6 @@
 import org.junit.jupiter.api.Test;
 
 public class MultiLevelIndexTest {
-  private static final SecureRandom random = new SecureRandom();
   private Configuration hadoopConf = new Configuration();
 
   @Test
@@ -120,7 +119,7 @@
     liter = reader.lookup(new Key(String.format("%05d000", num + 1)));
     assertFalse(liter.hasNext());
 
-    random.ints(100, 0, num * 1_000).forEach(k -> {
+    RANDOM.get().ints(100, 0, num * 1_000).forEach(k -> {
       int expected;
       if (k % 1000 == 0) {
         expected = k / 1000; // end key is inclusive
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
index ea0c4ce..2f1525d 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java
@@ -20,6 +20,7 @@
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -29,7 +30,6 @@
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.io.UncheckedIOException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -71,7 +71,6 @@
 @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
 public class MultiThreadedRFileTest {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Logger LOG = LoggerFactory.getLogger(MultiThreadedRFileTest.class);
   private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<>();
 
@@ -81,16 +80,16 @@
     if (indexIter.hasTop()) {
       Key lastKey = new Key(indexIter.getTopKey());
 
-      if (reader.getFirstKey().compareTo(lastKey) > 0) {
-        throw new RuntimeException(
-            "First key out of order " + reader.getFirstKey() + " " + lastKey);
+      if (reader.getFirstRow().compareTo(lastKey.getRow()) > 0) {
+        throw new IllegalStateException(
+            "First key out of order " + reader.getFirstRow() + " " + lastKey);
       }
 
       indexIter.next();
 
       while (indexIter.hasTop()) {
         if (lastKey.compareTo(indexIter.getTopKey()) > 0) {
-          throw new RuntimeException(
+          throw new IllegalStateException(
               "Indext out of order " + lastKey + " " + indexIter.getTopKey());
         }
 
@@ -99,8 +98,9 @@
 
       }
 
-      if (!reader.getLastKey().equals(lastKey)) {
-        throw new RuntimeException("Last key out of order " + reader.getLastKey() + " " + lastKey);
+      if (!reader.getLastRow().equals(lastKey.getRow())) {
+        throw new IllegalStateException(
+            "Last key out of order " + reader.getLastRow() + " " + lastKey);
       }
     }
   }
@@ -134,7 +134,7 @@
     public TestRFile deepCopy() throws IOException {
       TestRFile copy = new TestRFile(accumuloConfiguration);
       // does not copy any writer resources. This would be for read only.
-      copy.reader = (Reader) reader.deepCopy(null);
+      copy.reader = reader.deepCopy(null);
       copy.rfile = rfile;
       copy.iter = new ColumnFamilySkippingIterator(copy.reader);
       copy.deepCopy = true;
@@ -286,7 +286,7 @@
   }
 
   private void validate(TestRFile trf) throws IOException {
-    random.ints(10, 0, 4).forEach(part -> {
+    RANDOM.get().ints(10, 0, 4).forEach(part -> {
       try {
         Range range = new Range(getKey(part, 0, 0), true, getKey(part, 4, 2048), true);
         trf.iter.seek(range, EMPTY_COL_FAMS, false);
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
index 6a9b969..3dec555 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.core.file.rfile;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -33,7 +34,6 @@
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.security.SecureRandom;
 import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -41,7 +41,6 @@
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -49,7 +48,6 @@
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.Sampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-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;
@@ -65,16 +63,14 @@
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheManagerFactory;
-import org.apache.accumulo.core.file.blockfile.cache.lru.LruBlockCache;
 import org.apache.accumulo.core.file.blockfile.cache.lru.LruBlockCacheManager;
 import org.apache.accumulo.core.file.blockfile.impl.BasicCacheProvider;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.CachableBuilder;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
-import org.apache.accumulo.core.file.rfile.bcfile.BCFile;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.ColumnFamilySkippingIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+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.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -86,8 +82,6 @@
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.io.Text;
@@ -103,9 +97,7 @@
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-public class RFileTest {
-
-  private static final SecureRandom random = new SecureRandom();
+public class RFileTest extends AbstractRFileTest {
 
   public static class SampleIE implements IteratorEnvironment {
 
@@ -126,7 +118,6 @@
     }
   }
 
-  private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<>();
   private static final Configuration hadoopConf = new Configuration();
 
   @TempDir
@@ -198,154 +189,6 @@
 
   }
 
-  private static void checkIndex(Reader reader) throws IOException {
-    FileSKVIterator indexIter = reader.getIndex();
-
-    if (indexIter.hasTop()) {
-      Key lastKey = new Key(indexIter.getTopKey());
-
-      if (reader.getFirstKey().compareTo(lastKey) > 0) {
-        throw new RuntimeException(
-            "First key out of order " + reader.getFirstKey() + " " + lastKey);
-      }
-
-      indexIter.next();
-
-      while (indexIter.hasTop()) {
-        if (lastKey.compareTo(indexIter.getTopKey()) > 0) {
-          throw new RuntimeException(
-              "Indext out of order " + lastKey + " " + indexIter.getTopKey());
-        }
-
-        lastKey = new Key(indexIter.getTopKey());
-        indexIter.next();
-
-      }
-
-      if (!reader.getLastKey().equals(lastKey)) {
-        throw new RuntimeException("Last key out of order " + reader.getLastKey() + " " + lastKey);
-      }
-    }
-  }
-
-  public static class TestRFile {
-
-    protected Configuration conf = new Configuration();
-    public RFile.Writer writer;
-    protected ByteArrayOutputStream baos;
-    protected FSDataOutputStream dos;
-    protected SeekableByteArrayInputStream bais;
-    protected FSDataInputStream in;
-    protected AccumuloConfiguration accumuloConfiguration;
-    public Reader reader;
-    public SortedKeyValueIterator<Key,Value> iter;
-    private BlockCacheManager manager;
-
-    public TestRFile(AccumuloConfiguration accumuloConfiguration) {
-      this.accumuloConfiguration = accumuloConfiguration;
-      if (this.accumuloConfiguration == null) {
-        this.accumuloConfiguration = DefaultConfiguration.getInstance();
-      }
-    }
-
-    public void openWriter(boolean startDLG) throws IOException {
-      openWriter(startDLG, 1000);
-    }
-
-    public void openWriter(boolean startDLG, int blockSize) throws IOException {
-      baos = new ByteArrayOutputStream();
-      dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
-      CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE,
-          accumuloConfiguration.getAllCryptoProperties());
-
-      BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", conf, cs);
-
-      SamplerConfigurationImpl samplerConfig =
-          SamplerConfigurationImpl.newSamplerConfig(accumuloConfiguration);
-      Sampler sampler = null;
-
-      if (samplerConfig != null) {
-        sampler = SamplerFactory.newSampler(samplerConfig, accumuloConfiguration);
-      }
-
-      writer = new RFile.Writer(_cbw, blockSize, 1000, samplerConfig, sampler);
-
-      if (startDLG) {
-        writer.startDefaultLocalityGroup();
-      }
-    }
-
-    public void openWriter() throws IOException {
-      openWriter(1000);
-    }
-
-    public void openWriter(int blockSize) throws IOException {
-      openWriter(true, blockSize);
-    }
-
-    public void closeWriter() throws IOException {
-      dos.flush();
-      writer.close();
-      dos.close();
-      if (baos != null) {
-        baos.close();
-      }
-    }
-
-    public void openReader() throws IOException {
-      openReader(true);
-    }
-
-    public void openReader(boolean cfsi) throws IOException {
-      int fileLength = 0;
-      byte[] data = null;
-      data = baos.toByteArray();
-
-      bais = new SeekableByteArrayInputStream(data);
-      in = new FSDataInputStream(bais);
-      fileLength = data.length;
-
-      DefaultConfiguration dc = DefaultConfiguration.getInstance();
-      ConfigurationCopy cc = new ConfigurationCopy(dc);
-      cc.set(Property.TSERV_CACHE_MANAGER_IMPL, LruBlockCacheManager.class.getName());
-      try {
-        manager = BlockCacheManagerFactory.getInstance(cc);
-      } catch (Exception e) {
-        throw new RuntimeException("Error creating BlockCacheManager", e);
-      }
-      cc.set(Property.TSERV_DEFAULT_BLOCKSIZE, Long.toString(100000));
-      cc.set(Property.TSERV_DATACACHE_SIZE, Long.toString(100000000));
-      cc.set(Property.TSERV_INDEXCACHE_SIZE, Long.toString(100000000));
-      manager.start(BlockCacheConfiguration.forTabletServer(cc));
-      LruBlockCache indexCache = (LruBlockCache) manager.getBlockCache(CacheType.INDEX);
-      LruBlockCache dataCache = (LruBlockCache) manager.getBlockCache(CacheType.DATA);
-
-      CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE,
-          accumuloConfiguration.getAllCryptoProperties());
-
-      CachableBuilder cb = new CachableBuilder().input(in, "source-1").length(fileLength).conf(conf)
-          .cacheProvider(new BasicCacheProvider(indexCache, dataCache)).cryptoService(cs);
-      reader = new RFile.Reader(cb);
-      if (cfsi) {
-        iter = new ColumnFamilySkippingIterator(reader);
-      }
-
-      checkIndex(reader);
-    }
-
-    public void closeReader() throws IOException {
-      reader.close();
-      in.close();
-      if (null != manager) {
-        manager.stop();
-      }
-    }
-
-    public void seek(Key nk) throws IOException {
-      iter.seek(new Range(nk, null), EMPTY_COL_FAMS, false);
-    }
-  }
-
   static Key newKey(String row, String cf, String cq, String cv, long ts) {
     return new Key(row.getBytes(), cf.getBytes(), cq.getBytes(), cv.getBytes(), ts);
   }
@@ -358,8 +201,6 @@
     return String.format(prefix + "%06d", i);
   }
 
-  public AccumuloConfiguration conf = null;
-
   @Test
   public void test1() throws IOException {
 
@@ -374,7 +215,7 @@
     trf.iter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false);
     assertFalse(trf.iter.hasTop());
 
-    assertNull(trf.reader.getLastKey());
+    assertNull(trf.reader.getLastRow());
 
     trf.closeReader();
   }
@@ -411,7 +252,7 @@
     trf.iter.next();
     assertFalse(trf.iter.hasTop());
 
-    assertEquals(newKey("r1", "cf1", "cq1", "L1", 55), trf.reader.getLastKey());
+    assertEquals(new Text("r1"), trf.reader.getLastRow());
 
     trf.closeReader();
   }
@@ -547,12 +388,12 @@
       }
     }
 
-    assertEquals(expectedKeys.get(expectedKeys.size() - 1), trf.reader.getLastKey());
+    assertEquals(expectedKeys.get(expectedKeys.size() - 1).getRow(), trf.reader.getLastRow());
 
     // test seeking to random location and reading all data from that point
     // there was an off by one bug with this in the transient index
     for (int i = 0; i < 12; i++) {
-      index = random.nextInt(expectedKeys.size());
+      index = RANDOM.get().nextInt(expectedKeys.size());
       trf.seek(expectedKeys.get(index));
       for (; index < expectedKeys.size(); index++) {
         assertTrue(trf.iter.hasTop());
@@ -574,22 +415,6 @@
     trf.closeReader();
   }
 
-  private void verify(TestRFile trf, Iterator<Key> eki, Iterator<Value> evi) throws IOException {
-
-    while (trf.iter.hasTop()) {
-      Key ek = eki.next();
-      Value ev = evi.next();
-
-      assertEquals(ek, trf.iter.getTopKey());
-      assertEquals(ev, trf.iter.getTopValue());
-
-      trf.iter.next();
-    }
-
-    assertFalse(eki.hasNext());
-    assertFalse(evi.hasNext());
-  }
-
   @Test
   public void test4() throws IOException {
     TestRFile trf = new TestRFile(conf);
@@ -639,7 +464,7 @@
     assertEquals(newKey("r1", "cf1", "cq1", "L1", 55), trf.iter.getTopKey());
     assertEquals(newValue("foo1"), trf.iter.getTopValue());
 
-    assertEquals(newKey("r1", "cf1", "cq4", "L1", 56), trf.reader.getLastKey());
+    assertEquals(new Text("r1"), trf.reader.getLastRow());
 
     trf.closeReader();
   }
@@ -672,7 +497,7 @@
       assertFalse(trf.iter.hasTop());
     }
 
-    assertEquals(newKey(formatString("r_", 499), "cf1", "cq1", "L1", 55), trf.reader.getLastKey());
+    assertEquals(new Text(formatString("r_", 499)), trf.reader.getLastRow());
 
     trf.closeReader();
   }
@@ -738,7 +563,7 @@
         newKey(formatString("r_", 2), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false);
     assertFalse(trf.iter.hasTop());
 
-    assertEquals(newKey(formatString("r_", 49), "cf1", "cq1", "L1", 55), trf.reader.getLastKey());
+    assertEquals(new Text(formatString("r_", 49)), trf.reader.getLastRow());
 
     trf.reader.close();
   }
@@ -1645,13 +1470,13 @@
 
     for (int count = 0; count < 100; count++) {
 
-      int start = random.nextInt(2300);
+      int start = RANDOM.get().nextInt(2300);
       Range range = new Range(newKey(formatString("r_", start), "cf1", "cq1", "L1", 42),
           newKey(formatString("r_", start + 100), "cf1", "cq1", "L1", 42));
 
       trf.reader.seek(range, cfs, false);
 
-      int numToScan = random.nextInt(100);
+      int numToScan = RANDOM.get().nextInt(100);
 
       for (int j = 0; j < numToScan; j++) {
         assertTrue(trf.reader.hasTop());
@@ -1667,8 +1492,8 @@
       // seek a little forward from the last range and read a few keys within the unconsumed portion
       // of the last range
 
-      int start2 = start + numToScan + random.nextInt(3);
-      int end2 = start2 + random.nextInt(3);
+      int start2 = start + numToScan + RANDOM.get().nextInt(3);
+      int end2 = start2 + RANDOM.get().nextInt(3);
 
       range = new Range(newKey(formatString("r_", start2), "cf1", "cq1", "L1", 42),
           newKey(formatString("r_", end2), "cf1", "cq1", "L1", 42));
@@ -2004,24 +1829,24 @@
       boolean endInclusive = false;
       int endIndex = sampleData.size();
 
-      if (random.nextBoolean()) {
-        startIndex = random.nextInt(sampleData.size());
+      if (RANDOM.get().nextBoolean()) {
+        startIndex = RANDOM.get().nextInt(sampleData.size());
         startKey = sampleData.get(startIndex).getKey();
-        startInclusive = random.nextBoolean();
+        startInclusive = RANDOM.get().nextBoolean();
         if (!startInclusive) {
           startIndex++;
         }
       }
 
-      if (startIndex < endIndex && random.nextBoolean()) {
-        endIndex -= random.nextInt(endIndex - startIndex);
+      if (startIndex < endIndex && RANDOM.get().nextBoolean()) {
+        endIndex -= RANDOM.get().nextInt(endIndex - startIndex);
         endKey = sampleData.get(endIndex - 1).getKey();
-        endInclusive = random.nextBoolean();
+        endInclusive = RANDOM.get().nextBoolean();
         if (!endInclusive) {
           endIndex--;
         }
       } else if (startIndex == endIndex) {
-        endInclusive = random.nextBoolean();
+        endInclusive = RANDOM.get().nextBoolean();
       }
 
       sample.seek(new Range(startKey, startInclusive, endKey, endInclusive), columnFamilies,
@@ -2303,8 +2128,8 @@
 
     // mfw.startDefaultLocalityGroup();
 
-    Text tableExtent = new Text(
-        TabletsSection.encodeRow(MetadataTable.ID, TabletsSection.getRange().getEndKey().getRow()));
+    Text tableExtent = new Text(TabletsSection.encodeRow(AccumuloTable.METADATA.tableId(),
+        TabletsSection.getRange().getEndKey().getRow()));
 
     // table tablet's directory
     Key tableDirKey = new Key(tableExtent, ServerColumnFamily.DIRECTORY_COLUMN.getColumnFamily(),
@@ -2322,7 +2147,7 @@
     mfw.append(tablePrevRowKey, TabletColumnFamily.encodePrevEndRow(null));
 
     // ----------] default tablet info
-    Text defaultExtent = new Text(TabletsSection.encodeRow(MetadataTable.ID, null));
+    Text defaultExtent = new Text(TabletsSection.encodeRow(AccumuloTable.METADATA.tableId(), null));
 
     // default's directory
     Key defaultDirKey =
@@ -2356,7 +2181,7 @@
     testRfile.iter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false);
     assertTrue(testRfile.iter.hasTop());
 
-    assertNotNull(testRfile.reader.getLastKey());
+    assertNotNull(testRfile.reader.getLastRow());
 
     testRfile.closeReader();
 
diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/RollingStatsTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/RollingStatsTest.java
index e9a978a..6f91e56 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RollingStatsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RollingStatsTest.java
@@ -18,9 +18,9 @@
  */
 package org.apache.accumulo.core.file.rfile;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.security.SecureRandom;
 import java.util.function.IntSupplier;
 
 import org.apache.commons.math3.distribution.NormalDistribution;
@@ -34,7 +34,6 @@
 public class RollingStatsTest {
 
   private static final double TOLERANCE = 1.0 / 1000;
-  private static final SecureRandom random = new SecureRandom();
 
   private static void assertFuzzyEquals(double expected, double actual) {
     assertTrue(DoubleMath.fuzzyEquals(expected, actual, TOLERANCE), String.format(
@@ -80,7 +79,7 @@
       rsp.addValue(v);
       checkAgreement(ds, rs);
 
-      if (random.nextDouble() < 0.001) {
+      if (RANDOM.get().nextDouble() < 0.001) {
         checkAgreement(ds, rsp);
       }
     }
@@ -95,7 +94,7 @@
     StatTester st = new StatTester(1019);
     int[] keySizes = {103, 113, 123, 2345};
     for (int i = 0; i < 10000; i++) {
-      st.addValue(keySizes[random.nextInt(keySizes.length)]);
+      st.addValue(keySizes[RANDOM.get().nextInt(keySizes.length)]);
     }
     st.check();
   }
@@ -120,7 +119,7 @@
       StatTester st = new StatTester(windowSize);
 
       for (int i = 0; i < 1000; i++) {
-        int v = 200 + random.nextInt(50);
+        int v = 200 + RANDOM.get().nextInt(50);
 
         st.addValue(v);
       }
@@ -177,9 +176,9 @@
     for (int i = 0; i < 13; i++) {
 
       // write small keys
-      int numSmall = 1000 + random.nextInt(1000);
+      int numSmall = 1000 + RANDOM.get().nextInt(1000);
       for (int s = 0; s < numSmall; s++) {
-        int sks = 50 + random.nextInt(100);
+        int sks = 50 + RANDOM.get().nextInt(100);
         // simulate row with multiple cols
         for (int c = 0; c < 3; c++) {
           st.addValue(sks);
@@ -187,9 +186,9 @@
       }
 
       // write a few large keys
-      int numLarge = 1 + random.nextInt(1);
+      int numLarge = 1 + RANDOM.get().nextInt(1);
       for (int l = 0; l < numLarge; l++) {
-        int lks = 500000 + random.nextInt(1000000);
+        int lks = 500000 + RANDOM.get().nextInt(1000000);
         for (int c = 0; c < 3; c++) {
           st.addValue(lks);
         }
diff --git a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java
index 11936f8..8eb3a5e 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.file.streams;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.InputStream;
-import java.security.SecureRandom;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.util.ratelimit.RateLimiter;
@@ -31,8 +31,6 @@
 
 public class RateLimitedInputStreamTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
   @Test
   public void permitsAreProperlyAcquired() throws Exception {
     // Create variables for tracking behaviors of mock object
@@ -49,7 +47,7 @@
     long bytesRetrieved = 0;
     try (InputStream is = new RateLimitedInputStream(new RandomInputStream(), rateLimiter)) {
       for (int i = 0; i < 100; ++i) {
-        int count = Math.abs(random.nextInt()) % 65536;
+        int count = Math.abs(RANDOM.get().nextInt()) % 65536;
         int countRead = is.read(new byte[count]);
         assertEquals(count, countRead);
         bytesRetrieved += count;
@@ -62,7 +60,7 @@
 
     @Override
     public int read() {
-      return random.nextInt() & 0xff;
+      return RANDOM.get().nextInt() & 0xff;
     }
 
     @Override
diff --git a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java
index d83cff4..8df1a31 100644
--- a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.file.streams;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.io.OutputStream;
-import java.security.SecureRandom;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.util.ratelimit.RateLimiter;
@@ -33,8 +33,6 @@
 
 public class RateLimitedOutputStreamTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
   @Test
   public void permitsAreProperlyAcquired() throws Exception {
     // Create variables for tracking behaviors of mock object
@@ -52,7 +50,7 @@
     try (RateLimitedOutputStream os =
         new RateLimitedOutputStream(new NullOutputStream(), rateLimiter)) {
       for (int i = 0; i < 100; ++i) {
-        byte[] bytes = new byte[Math.abs(random.nextInt() % 65536)];
+        byte[] bytes = new byte[Math.abs(RANDOM.get().nextInt() % 65536)];
         os.write(bytes);
         bytesWritten += bytes.length;
       }
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/CombinerTestUtil.java b/core/src/test/java/org/apache/accumulo/core/iterators/CombinerTestUtil.java
index 18904cd..3956796 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/CombinerTestUtil.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/CombinerTestUtil.java
@@ -24,6 +24,6 @@
   }
 
   public static long cacheSize() {
-    return Combiner.loggedMsgCache.size();
+    return Combiner.loggedMsgCache.estimatedSize();
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java b/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
index 802dfe8..8c395a2 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
@@ -18,15 +18,9 @@
  */
 package org.apache.accumulo.core.iterators;
 
-import java.io.IOException;
-
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iteratorsImpl.system.MapFileIterator;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
 
 public class DefaultIteratorEnvironment implements IteratorEnvironment {
 
@@ -41,14 +35,6 @@
     this.conf = DefaultConfiguration.getInstance();
   }
 
-  @Deprecated(since = "2.0.0")
-  @Override
-  public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName)
-      throws IOException {
-    FileSystem fs = FileSystem.get(hadoopConf);
-    return new MapFileIterator(fs, mapFileName, hadoopConf);
-  }
-
   @Override
   public boolean isSamplingEnabled() {
     return false;
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
index 9c6bdb6..97ce48b 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -35,8 +35,8 @@
 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.file.rfile.AbstractRFileTest.TestRFile;
 import org.apache.accumulo.core.file.rfile.RFileTest;
-import org.apache.accumulo.core.file.rfile.RFileTest.TestRFile;
 import org.apache.accumulo.core.iterators.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -46,7 +46,6 @@
 
 public class IndexedDocIteratorTest {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<>();
   private static final byte[] nullByte = {0};
 
@@ -93,7 +92,7 @@
         doc.append(nullByte, 0, 1);
         doc.append(String.format("%010d", docid).getBytes(), 0, 10);
         for (int j = 0; j < columnFamilies.length; j++) {
-          if (random.nextFloat() < hitRatio) {
+          if (RANDOM.get().nextFloat() < hitRatio) {
             Text colq = new Text(columnFamilies[j]);
             colq.append(nullByte, 0, 1);
             colq.append(doc.getBytes(), 0, doc.getLength());
@@ -116,7 +115,7 @@
           docs.add(doc);
         }
         for (Text cf : otherColumnFamilies) {
-          if (random.nextFloat() < hitRatio) {
+          if (RANDOM.get().nextFloat() < hitRatio) {
             Text colq = new Text(cf);
             colq.append(nullByte, 0, 1);
             colq.append(doc.getBytes(), 0, doc.getLength());
@@ -149,7 +148,7 @@
   private SortedKeyValueIterator<Key,Value> createIteratorStack(float hitRatio, int numRows,
       int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies, HashSet<Text> docs,
       Text[] negatedColumns) throws IOException {
-    // write a map file
+    // write a data file
     trf.openWriter(false);
 
     TreeMap<Key,Value> inMemoryMap = createSortedMap(hitRatio, numRows, numDocsPerRow,
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
index 2c3710d..52c3a79 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -43,7 +43,6 @@
 
 public class IntersectingIteratorTest {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<>();
   private static IteratorEnvironment env = new DefaultIteratorEnvironment();
 
@@ -78,7 +77,7 @@
         boolean docHits = true;
         Text doc = new Text(String.format("%010d", docid));
         for (int j = 0; j < columnFamilies.length; j++) {
-          if (random.nextFloat() < hitRatio) {
+          if (RANDOM.get().nextFloat() < hitRatio) {
             Key k = new Key(row, columnFamilies[j], doc);
             map.put(k, v);
             if (negateMask[j]) {
@@ -94,7 +93,7 @@
           docs.add(doc);
         }
         for (Text cf : otherColumnFamilies) {
-          if (random.nextFloat() < hitRatio) {
+          if (RANDOM.get().nextFloat() < hitRatio) {
             Key k = new Key(row, cf, doc);
             map.put(k, v);
           }
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java
index 34d454e..02771e3 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TestCfCqSlice.java
@@ -19,10 +19,10 @@
 package org.apache.accumulo.core.iterators.user;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.security.SecureRandom;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -45,7 +45,6 @@
 
 public abstract class TestCfCqSlice {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Range INFINITY = new Range();
   private static final Lexicoder<Long> LONG_LEX = new ReadableLongLexicoder(4);
   private static final AtomicLong ROW_ID_GEN = new AtomicLong();
@@ -71,7 +70,7 @@
   }
 
   @Test
-  public void testAllRowsFullSlice() {
+  public void testAllRowsFullSlice() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     loadKvs(foundKvs, EMPTY_OPTS, INFINITY);
     for (int i = 0; i < LR_DIM; i++) {
@@ -85,7 +84,7 @@
   }
 
   @Test
-  public void testSingleRowFullSlice() {
+  public void testSingleRowFullSlice() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     int rowId = LR_DIM / 2;
     loadKvs(foundKvs, EMPTY_OPTS, Range.exact(new Text(LONG_LEX.encode((long) rowId))));
@@ -105,7 +104,7 @@
   }
 
   @Test
-  public void testAllRowsSlice() {
+  public void testAllRowsSlice() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCf = 20;
     long sliceMinCq = 30;
@@ -137,7 +136,7 @@
   }
 
   @Test
-  public void testSingleColumnSlice() {
+  public void testSingleColumnSlice() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCf = 20;
     long sliceMinCq = 20;
@@ -165,7 +164,7 @@
   }
 
   @Test
-  public void testSingleColumnSliceByExclude() {
+  public void testSingleColumnSliceByExclude() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCf = 20;
     long sliceMinCq = 20;
@@ -195,7 +194,7 @@
   }
 
   @Test
-  public void testAllCfsCqSlice() {
+  public void testAllCfsCqSlice() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCq = 10;
     long sliceMaxCq = 30;
@@ -219,7 +218,7 @@
   }
 
   @Test
-  public void testSliceCfsAllCqs() {
+  public void testSliceCfsAllCqs() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCf = 10;
     long sliceMaxCf = 30;
@@ -243,7 +242,7 @@
   }
 
   @Test
-  public void testEmptySlice() {
+  public void testEmptySlice() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCf = LR_DIM + 1;
     long sliceMinCq = LR_DIM + 1;
@@ -304,7 +303,7 @@
   }
 
   @Test
-  public void testSeekMinExclusive() {
+  public void testSeekMinExclusive() throws Exception {
     boolean[][][] foundKvs = new boolean[LR_DIM][LR_DIM][LR_DIM];
     long sliceMinCf = 20;
     long sliceMinCq = 30;
@@ -360,37 +359,33 @@
     }
   }
 
-  private void loadKvs(boolean[][][] foundKvs, Map<String,String> options, Range range) {
+  private void loadKvs(boolean[][][] foundKvs, Map<String,String> options, Range range)
+      throws Exception {
     loadKvs(new SortedMapIterator(data), foundKvs, options, range);
   }
 
   private void loadKvs(SortedKeyValueIterator<Key,Value> parent, boolean[][][] foundKvs,
-      Map<String,String> options, Range range) {
-    try {
-      SortedKeyValueIterator<Key,Value> skvi =
-          getFilterClass().getDeclaredConstructor().newInstance();
-      skvi.init(parent, options, null);
-      skvi.seek(range, EMPTY_CF_SET, false);
+      Map<String,String> options, Range range) throws Exception {
+    SortedKeyValueIterator<Key,Value> skvi =
+        getFilterClass().getDeclaredConstructor().newInstance();
+    skvi.init(parent, options, null);
+    skvi.seek(range, EMPTY_CF_SET, false);
 
-      while (skvi.hasTop()) {
-        Key k = skvi.getTopKey();
-        int row = LONG_LEX.decode(k.getRow().copyBytes()).intValue();
-        int cf = LONG_LEX.decode(k.getColumnFamily().copyBytes()).intValue();
-        int cq = LONG_LEX.decode(k.getColumnQualifier().copyBytes()).intValue();
+    while (skvi.hasTop()) {
+      Key k = skvi.getTopKey();
+      int row = LONG_LEX.decode(k.getRow().copyBytes()).intValue();
+      int cf = LONG_LEX.decode(k.getColumnFamily().copyBytes()).intValue();
+      int cq = LONG_LEX.decode(k.getColumnQualifier().copyBytes()).intValue();
 
-        assertFalse(foundKvs[row][cf][cq], "Duplicate " + row + " " + cf + " " + cq);
-        foundKvs[row][cf][cq] = true;
+      assertFalse(foundKvs[row][cf][cq], "Duplicate " + row + " " + cf + " " + cq);
+      foundKvs[row][cf][cq] = true;
 
-        if (random.nextInt(100) == 0) {
-          skvi.seek(new Range(k, false, range.getEndKey(), range.isEndKeyInclusive()), EMPTY_CF_SET,
-              false);
-        } else {
-          skvi.next();
-        }
+      if (RANDOM.get().nextInt(100) == 0) {
+        skvi.seek(new Range(k, false, range.getEndKey(), range.isEndKeyInclusive()), EMPTY_CF_SET,
+            false);
+      } else {
+        skvi.next();
       }
-
-    } catch (Exception e) {
-      throw new RuntimeException(e);
     }
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
index 6798b6f..1879f37 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
@@ -90,7 +90,7 @@
     try {
       titer = clazz.getDeclaredConstructor().newInstance();
     } catch (ReflectiveOperationException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
 
     IteratorEnvironment iterEnv = EasyMock.createMock(IteratorEnvironment.class);
diff --git a/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java
new file mode 100644
index 0000000..1abe862
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockDataTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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 org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Optional;
+import java.util.UUID;
+
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.net.HostAndPort;
+
+public class ServiceLockDataTest {
+
+  private final UUID serverUUID = UUID.randomUUID();
+
+  @Test
+  public void testSingleServiceConstructor() throws Exception {
+    ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV);
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV));
+    assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV));
+    assertEquals(ServiceDescriptor.DEFAULT_GROUP_NAME, ss.getGroup(ThriftService.TSERV));
+    assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddress(ThriftService.TABLET_SCAN));
+    assertNull(ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testMultipleServiceConstructor() throws Exception {
+    ServiceDescriptors sds = new ServiceDescriptors();
+    sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, "127.0.0.1:9997"));
+    sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, "127.0.0.1:9998"));
+    ServiceLockData ss = new ServiceLockData(sds);
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1:9997", ss.getAddressString(ThriftService.TSERV));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9997"), ss.getAddress(ThriftService.TSERV));
+    assertEquals(ServiceDescriptor.DEFAULT_GROUP_NAME, ss.getGroup(ThriftService.TSERV));
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertEquals("127.0.0.1:9998", ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9998"),
+        ss.getAddress(ThriftService.TABLET_SCAN));
+    assertEquals(ServiceDescriptor.DEFAULT_GROUP_NAME, ss.getGroup(ThriftService.TSERV));
+  }
+
+  @Test
+  public void testSingleServiceConstructorWithGroup() throws Exception {
+    ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, "meta");
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV));
+    assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV));
+    assertEquals("meta", ss.getGroup(ThriftService.TSERV));
+    assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddress(ThriftService.TABLET_SCAN));
+    assertNull(ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testSingleServiceConstructor2WithGroup() throws Exception {
+    ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, "meta");
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV));
+    assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV));
+    assertEquals("meta", ss.getGroup(ThriftService.TSERV));
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertNull(ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddress(ThriftService.TABLET_SCAN));
+    assertNull(ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testMultipleServiceConstructorWithGroup() throws Exception {
+    ServiceDescriptors sds = new ServiceDescriptors();
+    sds.addService(
+        new ServiceDescriptor(serverUUID, ThriftService.TSERV, "127.0.0.1:9997", "meta"));
+    sds.addService(
+        new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, "127.0.0.1:9998", "ns1"));
+    ServiceLockData ss = new ServiceLockData(sds);
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1:9997", ss.getAddressString(ThriftService.TSERV));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9997"), ss.getAddress(ThriftService.TSERV));
+    assertEquals("meta", ss.getGroup(ThriftService.TSERV));
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertEquals("127.0.0.1:9998", ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9998"),
+        ss.getAddress(ThriftService.TABLET_SCAN));
+    assertEquals("ns1", ss.getGroup(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddressString(ThriftService.COMPACTOR));
+    assertNull(ss.getAddress(ThriftService.COMPACTOR));
+    assertNull(ss.getGroup(ThriftService.COMPACTOR));
+  }
+
+  @Test
+  public void testParseEmpty() {
+    Optional<ServiceLockData> sld = ServiceLockData.parse(new byte[0]);
+    assertTrue(sld.isEmpty());
+    assertFalse(sld.isPresent());
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockTest.java b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockTest.java
similarity index 98%
rename from core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockTest.java
rename to core/src/test/java/org/apache/accumulo/core/lock/ServiceLockTest.java
index 44b6fcd..a5d1cd4 100644
--- a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ServiceLockTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/lock/ServiceLockTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.core.fate.zookeeper;
+package org.apache.accumulo.core.lock;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
index c5e1f2f..dbb4e45 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
@@ -40,19 +40,12 @@
   private static final TableId userTableId = TableId.of("tableId");
   private static ClientContext context;
 
-  @SuppressWarnings("deprecation")
-  private static final TableId REPL_TABLE_ID =
-      org.apache.accumulo.core.replication.ReplicationTable.ID;
-  @SuppressWarnings("deprecation")
-  private static final String REPL_TABLE_NAME =
-      org.apache.accumulo.core.replication.ReplicationTable.NAME;
-
   @BeforeAll
   public static void setupContext() {
     HashMap<String,String> tableNameToIdMap = new HashMap<>();
-    tableNameToIdMap.put(RootTable.NAME, RootTable.ID.canonical());
-    tableNameToIdMap.put(MetadataTable.NAME, MetadataTable.ID.canonical());
-    tableNameToIdMap.put(REPL_TABLE_NAME, REPL_TABLE_ID.canonical());
+    tableNameToIdMap.put(AccumuloTable.ROOT.tableName(), AccumuloTable.ROOT.tableId().canonical());
+    tableNameToIdMap.put(AccumuloTable.METADATA.tableName(),
+        AccumuloTable.METADATA.tableId().canonical());
     tableNameToIdMap.put(userTableName, userTableId.canonical());
 
     context = EasyMock.createMock(ClientContext.class);
@@ -64,56 +57,48 @@
 
   @Test
   public void checkSystemTableIdentifiers() {
-    assertNotEquals(RootTable.ID, MetadataTable.ID);
-    assertNotEquals(RootTable.NAME, MetadataTable.NAME);
+    assertNotEquals(AccumuloTable.ROOT.tableId(), AccumuloTable.METADATA.tableId());
+    assertNotEquals(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName());
   }
 
   @Test
   public void testGetCorrectServicer() throws AccumuloException, AccumuloSecurityException {
-    MetadataServicer ms = MetadataServicer.forTableId(context, RootTable.ID);
+    MetadataServicer ms = MetadataServicer.forTableId(context, AccumuloTable.ROOT.tableId());
     assertTrue(ms instanceof ServicerForRootTable);
     assertFalse(ms instanceof TableMetadataServicer);
-    assertEquals(RootTable.ID, ms.getServicedTableId());
+    assertEquals(AccumuloTable.ROOT.tableId(), ms.getServicedTableId());
 
-    ms = MetadataServicer.forTableId(context, MetadataTable.ID);
+    ms = MetadataServicer.forTableId(context, AccumuloTable.METADATA.tableId());
     assertTrue(ms instanceof ServicerForMetadataTable);
     assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(RootTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(MetadataTable.ID, ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableId(context, REPL_TABLE_ID);
-    assertTrue(ms instanceof ServicerForUserTables);
-    assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(MetadataTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(REPL_TABLE_ID, ms.getServicedTableId());
+    assertEquals(AccumuloTable.ROOT.tableName(),
+        ((TableMetadataServicer) ms).getServicingTableName());
+    assertEquals(AccumuloTable.METADATA.tableId(), ms.getServicedTableId());
 
     ms = MetadataServicer.forTableId(context, userTableId);
     assertTrue(ms instanceof ServicerForUserTables);
     assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(MetadataTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
+    assertEquals(AccumuloTable.METADATA.tableName(),
+        ((TableMetadataServicer) ms).getServicingTableName());
     assertEquals(userTableId, ms.getServicedTableId());
 
-    ms = MetadataServicer.forTableName(context, RootTable.NAME);
+    ms = MetadataServicer.forTableName(context, AccumuloTable.ROOT.tableName());
     assertTrue(ms instanceof ServicerForRootTable);
     assertFalse(ms instanceof TableMetadataServicer);
-    assertEquals(RootTable.ID, ms.getServicedTableId());
+    assertEquals(AccumuloTable.ROOT.tableId(), ms.getServicedTableId());
 
-    ms = MetadataServicer.forTableName(context, MetadataTable.NAME);
+    ms = MetadataServicer.forTableName(context, AccumuloTable.METADATA.tableName());
     assertTrue(ms instanceof ServicerForMetadataTable);
     assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(RootTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(MetadataTable.ID, ms.getServicedTableId());
-
-    ms = MetadataServicer.forTableName(context, REPL_TABLE_NAME);
-    assertTrue(ms instanceof ServicerForUserTables);
-    assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(MetadataTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
-    assertEquals(REPL_TABLE_ID, ms.getServicedTableId());
+    assertEquals(AccumuloTable.ROOT.tableName(),
+        ((TableMetadataServicer) ms).getServicingTableName());
+    assertEquals(AccumuloTable.METADATA.tableId(), ms.getServicedTableId());
 
     ms = MetadataServicer.forTableName(context, userTableName);
     assertTrue(ms instanceof ServicerForUserTables);
     assertTrue(ms instanceof TableMetadataServicer);
-    assertEquals(MetadataTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
+    assertEquals(AccumuloTable.METADATA.tableName(),
+        ((TableMetadataServicer) ms).getServicingTableName());
     assertEquals(userTableId, ms.getServicedTableId());
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/StoredTabletFileTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/StoredTabletFileTest.java
new file mode 100644
index 0000000..998c9ea
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/StoredTabletFileTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import org.junit.jupiter.api.Test;
+
+public class StoredTabletFileTest {
+
+  @Test
+  public void fileConversionTest() {
+    String s21 = "hdfs://localhost:8020/accumulo/tables/1/t-0000000/A000003v.rf";
+    String s31 =
+        "{\"path\":\"hdfs://localhost:8020/accumulo/tables/1/t-0000000/A000003v.rf\",\"startRow\":\"\",\"endRow\":\"\"}";
+    String s31_untrimmed =
+        "   {  \"path\":\"hdfs://localhost:8020/accumulo/tables/1/t-0000000/A000003v.rf\",\"startRow\":\"\",\"endRow\":\"\"  }   ";
+
+    assertTrue(StoredTabletFile.fileNeedsConversion(s21));
+    assertFalse(StoredTabletFile.fileNeedsConversion(s31));
+    assertFalse(StoredTabletFile.fileNeedsConversion(s31_untrimmed));
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/ReferencedTabletFileTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/ReferencedTabletFileTest.java
new file mode 100644
index 0000000..93e7bc0
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/schema/ReferencedTabletFileTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.schema;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+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.metadata.ReferencedTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+
+public class ReferencedTabletFileTest {
+
+  private ReferencedTabletFile test(String metadataFile, String volume, String tableId,
+      String tabletDir, String fileName) {
+    String metadataPath = StoredTabletFile.serialize(metadataFile);
+    StoredTabletFile storedTabletFile = new StoredTabletFile(metadataPath);
+    ReferencedTabletFile tabletFile = storedTabletFile.getTabletFile();
+
+    // Make sure original file name wasn't changed when serialized
+    assertTrue(metadataPath.contains(metadataFile));
+    assertEquals(volume, tabletFile.getVolume());
+    assertEquals(metadataPath, storedTabletFile.getMetadata());
+    assertEquals(TableId.of(tableId), tabletFile.getTableId());
+    assertEquals(tabletDir, tabletFile.getTabletDir());
+    assertEquals(fileName, tabletFile.getFileName());
+    return tabletFile;
+  }
+
+  @Test
+  public void testValidPaths() {
+    test("hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
+        "hdfs://localhost:8020/accumulo", "2a", "default_tablet", "F0000070.rf");
+    test("hdfs://nn1:9000/accumulo/tables/5a/t-0005/C0009.rf", "hdfs://nn1:9000/accumulo", "5a",
+        "t-0005", "C0009.rf");
+    test(
+        "file:/home/dude/workspace/accumulo/test/target/mini-tests/org.apache.accumulo.test.VolumeIT_test/volumes/v1/tables/1/t-0000003/F0000006.rf",
+        "file:/home/dude/workspace/accumulo/test/target/mini-tests/org.apache.accumulo.test.VolumeIT_test/volumes/v1",
+        "1", "t-0000003", "F0000006.rf");
+  }
+
+  @Test
+  public void testBadPaths() {
+    // 2a< srv:dir
+    final String message = "Failed to throw error on bad path";
+
+    assertThrows(IllegalArgumentException.class,
+        () -> test("C0004.rf", "", "2a", "t-0003", "C0004.rf"), message);
+    assertThrows(IllegalArgumentException.class, () -> test("dir", "", "2a", "", ""), message);
+
+    assertThrows(IllegalArgumentException.class,
+        () -> test("hdfs://localhost:8020/accumulo/tablets/2a/default_tablet/F0000070.rf",
+            "hdfs://localhost:8020/accumulo", "2a", "default_tablet", "F0000070.rf"),
+        message);
+    assertThrows(IllegalArgumentException.class,
+        () -> test("hdfs://localhost:8020/accumulo/2a/default_tablet/F0000070.rf",
+            " hdfs://localhost:8020/accumulo", "2a", "default_tablet", " F0000070.rf"),
+        message);
+    assertThrows(IllegalArgumentException.class,
+        () -> test("/accumulo/tables/2a/default_tablet/F0000070.rf", "", "2a", "default_tablet",
+            "F0000070.rf"),
+        message);
+    assertThrows(IllegalArgumentException.class,
+        () -> test("hdfs://localhost:8020/accumulo/tables/2a/F0000070.rf",
+            "hdfs://localhost:8020/accumulo", "2a", "", "F0000070.rf"),
+        message);
+    assertThrows(IllegalArgumentException.class,
+        () -> test("hdfs://localhost:8020/accumulo/tables/F0000070.rf",
+            "hdfs://localhost:8020/accumulo", null, "", "F0000070.rf"),
+        message);
+
+  }
+
+  private final String id = "2a";
+  private final String dir = "t-0003";
+  private final String filename = "C0004.rf";
+
+  @Test
+  public void testFullPathWithVolume() {
+    String volume = "hdfs://1.2.3.4/accumulo";
+    String metadataEntry = volume + "/tables/" + id + "/" + dir + "/" + filename;
+    test(metadataEntry, volume, id, dir, filename);
+  }
+
+  @Test
+  public void testNormalizePath() {
+    String uglyVolume = "hdfs://nn.somewhere.com:86753/accumulo/blah/.././/bad/bad2/../.././/////";
+    String metadataEntry = uglyVolume + "/tables/" + id + "/" + dir + "/" + filename;
+    ReferencedTabletFile uglyFile =
+        test(metadataEntry, "hdfs://nn.somewhere.com:86753/accumulo", id, dir, filename);
+    ReferencedTabletFile niceFile = StoredTabletFile
+        .of(new Path(
+            "hdfs://nn.somewhere.com:86753/accumulo/tables/" + id + "/" + dir + "/" + filename))
+        .getTabletFile();
+    assertEquals(niceFile, uglyFile);
+    assertEquals(niceFile.hashCode(), uglyFile.hashCode());
+  }
+
+  @Test
+  public void testNonRowRange() {
+    Path testPath = new Path("hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf");
+
+    // range where start key is not a row key
+    Range r1 = new Range(new Key("r1", "f1"), true, null, false);
+    assertThrows(IllegalArgumentException.class, () -> new ReferencedTabletFile(testPath, r1));
+
+    // range where end key is not a row key
+    Range r2 = new Range(null, true, new Key("r1", "f1"), false);
+    assertThrows(IllegalArgumentException.class, () -> new ReferencedTabletFile(testPath, r2));
+
+    // range where the key looks like a row, but the start key inclusivity is not whats expected
+    Range r3 = new Range(new Key("r1"), false, new Key("r2"), false);
+    assertThrows(IllegalArgumentException.class, () -> new ReferencedTabletFile(testPath, r3));
+
+    // range where the key looks like a row, but the end key inclusivity is not whats expected
+    Range r4 = new Range(new Key("r1"), true, new Key("r2"), true);
+    assertThrows(IllegalArgumentException.class, () -> new ReferencedTabletFile(testPath, r4));
+
+    // range where end key does not end with correct byte and is marked exclusive false
+    Range r5 = new Range(new Key("r1"), true, new Key("r2"), false);
+    assertThrows(IllegalArgumentException.class, () -> new ReferencedTabletFile(testPath, r5));
+
+    // This is valid as the end key is exclusive and ends in 0x00
+    Range r6 = new Range(new Key("r1"), true, new Key("r2").followingKey(PartialKey.ROW), false);
+    assertTrue(new ReferencedTabletFile(testPath, r6).hasRange());
+
+    // This is valid as the end key will be converted to exclusive and 0x00 should be appended
+    Range r7 = new Range(new Text("r1"), true, new Text("r2"), true);
+    assertTrue(new ReferencedTabletFile(testPath, r7).hasRange());
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java
deleted file mode 100644
index 9ac3846..0000000
--- a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java
+++ /dev/null
@@ -1,110 +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.schema;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.junit.jupiter.api.Test;
-
-public class TabletFileTest {
-
-  private TabletFile test(String metadataEntry, String volume, String tableId, String tabletDir,
-      String fileName) {
-    StoredTabletFile tabletFile = new StoredTabletFile(metadataEntry);
-
-    assertEquals(volume, tabletFile.getVolume());
-    assertEquals(metadataEntry, tabletFile.getMetaUpdateDelete());
-    assertEquals(TableId.of(tableId), tabletFile.getTableId());
-    assertEquals(tabletDir, tabletFile.getTabletDir());
-    assertEquals(fileName, tabletFile.getFileName());
-    return tabletFile;
-  }
-
-  @Test
-  public void testValidPaths() {
-    test("hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
-        "hdfs://localhost:8020/accumulo", "2a", "default_tablet", "F0000070.rf");
-    test("hdfs://nn1:9000/accumulo/tables/5a/t-0005/C0009.rf", "hdfs://nn1:9000/accumulo", "5a",
-        "t-0005", "C0009.rf");
-    test(
-        "file:/home/dude/workspace/accumulo/test/target/mini-tests/org.apache.accumulo.test.VolumeIT_test/volumes/v1/tables/1/t-0000003/F0000006.rf",
-        "file:/home/dude/workspace/accumulo/test/target/mini-tests/org.apache.accumulo.test.VolumeIT_test/volumes/v1",
-        "1", "t-0000003", "F0000006.rf");
-  }
-
-  @Test
-  public void testBadPaths() {
-    // 2a< srv:dir
-    final String message = "Failed to throw error on bad path";
-
-    assertThrows(IllegalArgumentException.class,
-        () -> test("C0004.rf", "", "2a", "t-0003", "C0004.rf"), message);
-    assertThrows(IllegalArgumentException.class, () -> test("dir", "", "2a", "", ""), message);
-
-    assertThrows(IllegalArgumentException.class,
-        () -> test("hdfs://localhost:8020/accumulo/tablets/2a/default_tablet/F0000070.rf",
-            "hdfs://localhost:8020/accumulo", "2a", "default_tablet", "F0000070.rf"),
-        message);
-    assertThrows(IllegalArgumentException.class,
-        () -> test("hdfs://localhost:8020/accumulo/2a/default_tablet/F0000070.rf",
-            " hdfs://localhost:8020/accumulo", "2a", "default_tablet", " F0000070.rf"),
-        message);
-    assertThrows(IllegalArgumentException.class,
-        () -> test("/accumulo/tables/2a/default_tablet/F0000070.rf", "", "2a", "default_tablet",
-            "F0000070.rf"),
-        message);
-    assertThrows(IllegalArgumentException.class,
-        () -> test("hdfs://localhost:8020/accumulo/tables/2a/F0000070.rf",
-            "hdfs://localhost:8020/accumulo", "2a", "", "F0000070.rf"),
-        message);
-    assertThrows(IllegalArgumentException.class,
-        () -> test("hdfs://localhost:8020/accumulo/tables/F0000070.rf",
-            "hdfs://localhost:8020/accumulo", null, "", "F0000070.rf"),
-        message);
-
-  }
-
-  private final String id = "2a";
-  private final String dir = "t-0003";
-  private final String filename = "C0004.rf";
-
-  @Test
-  public void testFullPathWithVolume() {
-    String volume = "hdfs://1.2.3.4/accumulo";
-    String metadataEntry = volume + "/tables/" + id + "/" + dir + "/" + filename;
-    test(metadataEntry, volume, id, dir, filename);
-  }
-
-  @Test
-  public void testNormalizePath() {
-    String uglyVolume = "hdfs://nn.somewhere.com:86753/accumulo/blah/.././/bad/bad2/../.././/////";
-    String metadataEntry = uglyVolume + "/tables/" + id + "/" + dir + "/" + filename;
-    TabletFile uglyFile =
-        test(metadataEntry, "hdfs://nn.somewhere.com:86753/accumulo", id, dir, filename);
-    TabletFile niceFile = new StoredTabletFile(
-        "hdfs://nn.somewhere.com:86753/accumulo/tables/" + id + "/" + dir + "/" + filename);
-    assertEquals(niceFile, uglyFile);
-    assertEquals(niceFile.hashCode(), uglyFile.hashCode());
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
index 7d9284b..3b2b4a8 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
@@ -19,11 +19,16 @@
 package org.apache.accumulo.core.metadata.schema;
 
 import static java.util.stream.Collectors.toSet;
-import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
+import static org.apache.accumulo.core.metadata.StoredTabletFile.serialize;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.MergedColumnFamily.MERGED_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.MergedColumnFamily.MERGED_VALUE;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn.SUSPEND_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LAST;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
@@ -42,6 +47,8 @@
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Stream;
 
@@ -51,6 +58,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.FateTxId;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.SuspendingTServer;
 import org.apache.accumulo.core.metadata.TServerInstance;
@@ -59,17 +67,25 @@
 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;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 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.ScanFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Builder;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+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.compaction.CompactionExecutorIdImpl;
+import org.apache.accumulo.core.util.time.SteadyTime;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class TabletMetadataTest {
 
   @Test
@@ -83,37 +99,53 @@
     FLUSH_COLUMN.put(mutation, new Value("6"));
     TIME_COLUMN.put(mutation, new Value("M123456789"));
 
-    String bf1 = "hdfs://nn1/acc/tables/1/t-0001/bf1";
-    String bf2 = "hdfs://nn1/acc/tables/1/t-0001/bf2";
+    String bf1 = serialize("hdfs://nn1/acc/tables/1/t-0001/bf1");
+    String bf2 = serialize("hdfs://nn1/acc/tables/1/t-0001/bf2");
     mutation.at().family(BulkFileColumnFamily.NAME).qualifier(bf1).put(FateTxId.formatTid(56));
     mutation.at().family(BulkFileColumnFamily.NAME).qualifier(bf2).put(FateTxId.formatTid(59));
 
     mutation.at().family(ClonedColumnFamily.NAME).qualifier("").put("OK");
 
     DataFileValue dfv1 = new DataFileValue(555, 23);
-    StoredTabletFile tf1 = new StoredTabletFile("hdfs://nn1/acc/tables/1/t-0001/df1.rf");
-    StoredTabletFile tf2 = new StoredTabletFile("hdfs://nn1/acc/tables/1/t-0001/df2.rf");
-    mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf1.getMetaUpdateDelete())
-        .put(dfv1.encode());
+    StoredTabletFile tf1 = StoredTabletFile.of(new Path("hdfs://nn1/acc/tables/1/t-0001/df1.rf"));
+    StoredTabletFile tf2 = StoredTabletFile.of(new Path("hdfs://nn1/acc/tables/1/t-0001/df2.rf"));
+    mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf1.getMetadata()).put(dfv1.encode());
     DataFileValue dfv2 = new DataFileValue(234, 13);
-    mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf2.getMetaUpdateDelete())
-        .put(dfv2.encode());
+    mutation.at().family(DataFileColumnFamily.NAME).qualifier(tf2.getMetadata()).put(dfv2.encode());
 
     mutation.at().family(CurrentLocationColumnFamily.NAME).qualifier("s001").put("server1:8555");
 
     mutation.at().family(LastLocationColumnFamily.NAME).qualifier("s000").put("server2:8555");
 
-    LogEntry le1 = new LogEntry(extent, 55, "lf1");
-    mutation.at().family(le1.getColumnFamily()).qualifier(le1.getColumnQualifier())
-        .timestamp(le1.timestamp).put(le1.getValue());
-    LogEntry le2 = new LogEntry(extent, 57, "lf2");
-    mutation.at().family(le2.getColumnFamily()).qualifier(le2.getColumnQualifier())
-        .timestamp(le2.timestamp).put(le2.getValue());
+    LogEntry le1 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID());
+    le1.addToMutation(mutation);
+    LogEntry le2 = LogEntry.fromPath("localhost+8020/" + UUID.randomUUID());
+    le2.addToMutation(mutation);
 
-    StoredTabletFile sf1 = new StoredTabletFile("hdfs://nn1/acc/tables/1/t-0001/sf1.rf");
-    StoredTabletFile sf2 = new StoredTabletFile("hdfs://nn1/acc/tables/1/t-0001/sf2.rf");
-    mutation.at().family(ScanFileColumnFamily.NAME).qualifier(sf1.getMetaUpdateDelete()).put("");
-    mutation.at().family(ScanFileColumnFamily.NAME).qualifier(sf2.getMetaUpdateDelete()).put("");
+    StoredTabletFile sf1 = StoredTabletFile.of(new Path("hdfs://nn1/acc/tables/1/t-0001/sf1.rf"));
+    StoredTabletFile sf2 = StoredTabletFile.of(new Path("hdfs://nn1/acc/tables/1/t-0001/sf2.rf"));
+    mutation.at().family(ScanFileColumnFamily.NAME).qualifier(sf1.getMetadata()).put("");
+    mutation.at().family(ScanFileColumnFamily.NAME).qualifier(sf2.getMetadata()).put("");
+
+    MERGED_COLUMN.put(mutation, new Value());
+
+    OLD_PREV_ROW_COLUMN.put(mutation, TabletColumnFamily.encodePrevEndRow(new Text("oldPrev")));
+    SteadyTime suspensionTime = SteadyTime.from(System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+    TServerInstance ser1 = new TServerInstance(HostAndPort.fromParts("server1", 8555), "s001");
+    Value suspend = SuspendingTServer.toValue(ser1, suspensionTime);
+    SUSPEND_COLUMN.put(mutation, suspend);
+    double splitRatio = .3;
+    SPLIT_RATIO_COLUMN.put(mutation, new Value(Double.toString(splitRatio)));
+
+    ExternalCompactionId ecid = ExternalCompactionId.generate(UUID.randomUUID());
+    ReferencedTabletFile tmpFile =
+        ReferencedTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/c1.rf"));
+    CompactionExecutorId ceid = CompactionExecutorIdImpl.externalId("G1");
+    Set<StoredTabletFile> jobFiles =
+        Set.of(StoredTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/b2.rf")));
+    ExternalCompactionMetadata ecMeta = new ExternalCompactionMetadata(jobFiles, jobFiles, tmpFile,
+        "localhost:4444", CompactionKind.SYSTEM, (short) 2, ceid, false, false, 44L);
+    mutation.put(ExternalCompactionColumnFamily.STR_NAME, ecid.canonical(), ecMeta.toJson());
 
     SortedMap<Key,Value> rowMap = toRowMap(mutation);
 
@@ -138,13 +170,18 @@
     assertEquals(HostAndPort.fromParts("server2", 8555), tm.getLast().getHostAndPort());
     assertEquals("s000", tm.getLast().getSession());
     assertEquals(LocationType.LAST, tm.getLast().getType());
-    assertEquals(Set.of(le1.getValue() + " " + le1.timestamp, le2.getValue() + " " + le2.timestamp),
-        tm.getLogs().stream().map(le -> le.getValue() + " " + le.timestamp).collect(toSet()));
+    assertEquals(Set.of(le1, le2), tm.getLogs().stream().collect(toSet()));
     assertEquals(extent.prevEndRow(), tm.getPrevEndRow());
     assertEquals(extent.tableId(), tm.getTableId());
     assertTrue(tm.sawPrevEndRow());
     assertEquals("M123456789", tm.getTime().encode());
     assertEquals(Set.of(sf1, sf2), Set.copyOf(tm.getScans()));
+    assertTrue(tm.hasMerged());
+    assertEquals(new Text("oldPrev"), tm.getOldPrevEndRow());
+    assertTrue(tm.sawOldPrevEndRow());
+    assertEquals(SuspendingTServer.fromValue(suspend), tm.getSuspend());
+    assertEquals(splitRatio, tm.getSplitRatio());
+    assertEquals(ecMeta.toJson(), tm.getExternalCompactions().get(ecid).toJson());
   }
 
   @Test
@@ -253,21 +290,45 @@
 
     // test SUSPENDED
     mutation = TabletColumnFamily.createPrevRowMutation(extent);
-    mutation.at().family(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily())
-        .qualifier(SuspendLocationColumn.SUSPEND_COLUMN.getColumnQualifier())
-        .put(SuspendingTServer.toValue(ser2, 1000L));
+    mutation.at().family(SUSPEND_COLUMN.getColumnFamily())
+        .qualifier(SUSPEND_COLUMN.getColumnQualifier())
+        .put(SuspendingTServer.toValue(ser2, SteadyTime.from(1000L, TimeUnit.MILLISECONDS)));
     rowMap = toRowMap(mutation);
 
     tm = TabletMetadata.convertRow(rowMap.entrySet().iterator(), colsToFetch, false);
 
     assertEquals(TabletState.SUSPENDED, tm.getTabletState(tservers));
-    assertEquals(1000L, tm.getSuspend().suspensionTime);
+    assertEquals(1000L, tm.getSuspend().suspensionTime.getMillis());
     assertEquals(ser2.getHostAndPort(), tm.getSuspend().server);
     assertNull(tm.getLocation());
     assertFalse(tm.hasCurrent());
   }
 
   @Test
+  public void testMergedColumn() {
+    KeyExtent extent = new KeyExtent(TableId.of("5"), new Text("df"), new Text("da"));
+
+    // Test merged column set
+    Mutation mutation = TabletColumnFamily.createPrevRowMutation(extent);
+    MERGED_COLUMN.put(mutation, MERGED_VALUE);
+    TabletMetadata tm = TabletMetadata.convertRow(toRowMap(mutation).entrySet().iterator(),
+        EnumSet.of(ColumnType.MERGED), true);
+    assertTrue(tm.hasMerged());
+
+    // Column not set
+    mutation = TabletColumnFamily.createPrevRowMutation(extent);
+    tm = TabletMetadata.convertRow(toRowMap(mutation).entrySet().iterator(),
+        EnumSet.of(ColumnType.MERGED), true);
+    assertFalse(tm.hasMerged());
+
+    // MERGED Column not fetched
+    mutation = TabletColumnFamily.createPrevRowMutation(extent);
+    tm = TabletMetadata.convertRow(toRowMap(mutation).entrySet().iterator(),
+        EnumSet.of(ColumnType.PREV_ROW), true);
+    assertThrows(IllegalStateException.class, tm::hasMerged);
+  }
+
+  @Test
   public void testTabletsMetadataAutoClose() throws Exception {
     AtomicBoolean closeCalled = new AtomicBoolean();
     AutoCloseable autoCloseable = () -> closeCalled.set(true);
@@ -288,6 +349,69 @@
     assertTrue(closeCalled.get());
   }
 
+  @Test
+  public void testTmBuilderImmutable() {
+    TabletMetadata.Builder b = new Builder();
+    var tm = b.build(EnumSet.allOf(ColumnType.class));
+
+    ExternalCompactionId ecid = ExternalCompactionId.generate(UUID.randomUUID());
+    ReferencedTabletFile tmpFile =
+        ReferencedTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/c1.rf"));
+    CompactionExecutorId ceid = CompactionExecutorIdImpl.externalId("G1");
+    StoredTabletFile stf = StoredTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/b2.rf"));
+    Set<StoredTabletFile> jobFiles = Set.of(stf);
+    ExternalCompactionMetadata ecMeta = new ExternalCompactionMetadata(jobFiles, jobFiles, tmpFile,
+        "localhost:4444", CompactionKind.SYSTEM, (short) 2, ceid, false, false, 44L);
+
+    // Verify the various collections are immutable and non-null (except for getKeyValues) if
+    // nothing set on the builder
+    assertTrue(tm.getExternalCompactions().isEmpty());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm.getExternalCompactions().put(ecid, ecMeta));
+    assertTrue(tm.getFiles().isEmpty());
+    assertTrue(tm.getFilesMap().isEmpty());
+    assertThrows(UnsupportedOperationException.class, () -> tm.getFiles().add(stf));
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm.getFilesMap().put(stf, new DataFileValue(0, 0, 0)));
+    assertTrue(tm.getLogs().isEmpty());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm.getLogs().add(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID())));
+    assertTrue(tm.getScans().isEmpty());
+    assertThrows(UnsupportedOperationException.class, () -> tm.getScans().add(stf));
+    assertTrue(tm.getLoaded().isEmpty());
+    assertThrows(UnsupportedOperationException.class, () -> tm.getLoaded().put(stf, 0L));
+    assertThrows(IllegalStateException.class, tm::getKeyValues);
+
+    // Set some data in the collections and very they are not empty but still immutable
+    b.extCompaction(ecid, ecMeta);
+    b.file(stf, new DataFileValue(0, 0, 0));
+    b.log(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID()));
+    b.scan(stf);
+    b.loadedFile(stf, 0L);
+    b.keyValue(new Key(), new Value());
+    var tm2 = b.build(EnumSet.allOf(ColumnType.class));
+
+    assertEquals(1, tm2.getExternalCompactions().size());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getExternalCompactions().put(ecid, ecMeta));
+    assertEquals(1, tm2.getFiles().size());
+    assertEquals(1, tm2.getFilesMap().size());
+    assertThrows(UnsupportedOperationException.class, () -> tm2.getFiles().add(stf));
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getFilesMap().put(stf, new DataFileValue(0, 0, 0)));
+    assertEquals(1, tm2.getLogs().size());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getLogs().add(LogEntry.fromPath("localhost+8020/" + UUID.randomUUID())));
+    assertEquals(1, tm2.getScans().size());
+    assertThrows(UnsupportedOperationException.class, () -> tm2.getScans().add(stf));
+    assertEquals(1, tm2.getLoaded().size());
+    assertThrows(UnsupportedOperationException.class, () -> tm2.getLoaded().put(stf, 0L));
+    assertEquals(1, tm2.getKeyValues().size());
+    assertThrows(UnsupportedOperationException.class,
+        () -> tm2.getKeyValues().put(new Key(), new Value()));
+
+  }
+
   private SortedMap<Key,Value> toRowMap(Mutation mutation) {
     SortedMap<Key,Value> rowMap = new TreeMap<>();
     mutation.getUpdates().forEach(cu -> {
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java
deleted file mode 100644
index bd2c20a..0000000
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationConfigurationUtilTest.java
+++ /dev/null
@@ -1,93 +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 org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-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.RootTable;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class ReplicationConfigurationUtilTest {
-
-  private AccumuloConfiguration conf;
-
-  @BeforeEach
-  public void setupConfiguration() {
-    Map<String,String> map = new HashMap<>();
-    map.put(Property.TABLE_REPLICATION.getKey(), "true");
-    conf = new ConfigurationCopy(map);
-  }
-
-  @Test
-  public void rootTableExtent() {
-    KeyExtent extent = new KeyExtent(RootTable.ID, null, null);
-    assertFalse(ReplicationConfigurationUtil.isEnabled(extent, conf),
-        "The root table should never be replicated");
-  }
-
-  @Test
-  public void metadataTableExtent() {
-    KeyExtent extent = new KeyExtent(MetadataTable.ID, null, null);
-    assertFalse(ReplicationConfigurationUtil.isEnabled(extent, conf),
-        "The metadata table should never be replicated");
-  }
-
-  @Test
-  public void rootTableExtentEmptyConf() {
-    KeyExtent extent = new KeyExtent(RootTable.ID, null, null);
-    assertFalse(
-        ReplicationConfigurationUtil.isEnabled(extent, new ConfigurationCopy(new HashMap<>())),
-        "The root table should never be replicated");
-  }
-
-  @Test
-  public void metadataTableExtentEmptyConf() {
-    KeyExtent extent = new KeyExtent(MetadataTable.ID, null, null);
-    assertFalse(
-        ReplicationConfigurationUtil.isEnabled(extent, new ConfigurationCopy(new HashMap<>())),
-        "The metadata table should never be replicated");
-  }
-
-  @Test
-  public void regularTable() {
-    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a"));
-    assertTrue(ReplicationConfigurationUtil.isEnabled(extent, conf), "Table should be replicated");
-  }
-
-  @Test
-  public void regularNonEnabledTable() {
-    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a"));
-    assertFalse(
-        ReplicationConfigurationUtil.isEnabled(extent, new ConfigurationCopy(new HashMap<>())),
-        "Table should not be replicated");
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
deleted file mode 100644
index bdfaf6b..0000000
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
+++ /dev/null
@@ -1,138 +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 org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-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.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class ReplicationSchemaTest {
-
-  @Test
-  public void extractFile() {
-    String file = "hdfs://foo:8020/bar";
-    Key k = new Key(new Text(file), StatusSection.NAME);
-    Text extractedFile = new Text();
-    StatusSection.getFile(k, extractedFile);
-    assertEquals(file, extractedFile.toString());
-  }
-
-  @Test
-  public void failOnNullKeyForFileExtract() {
-    Text extractedFile = new Text();
-    assertThrows(NullPointerException.class, () -> StatusSection.getFile(null, extractedFile));
-  }
-
-  @Test
-  public void failOnNullBufferForFileExtract() {
-    String file = "hdfs://foo:8020/bar";
-    Key k = new Key(file);
-    Text extractedFile = null;
-    assertThrows(NullPointerException.class, () -> StatusSection.getFile(k, extractedFile));
-  }
-
-  @Test
-  public void failOnExtractEmptyFile() {
-    String file = "";
-    Key k = new Key(file);
-    Text extractedFile = new Text();
-    assertThrows(IllegalArgumentException.class, () -> StatusSection.getFile(k, extractedFile));
-    assertEquals(file, extractedFile.toString());
-  }
-
-  @Test
-  public void extractTableId() {
-    TableId tableId = TableId.of("1");
-    Key k = new Key(new Text("foo"), StatusSection.NAME, new Text(tableId.canonical()));
-    assertEquals(tableId, StatusSection.getTableId(k));
-  }
-
-  @Test
-  public void extractTableIdUsingText() {
-    TableId tableId = TableId.of("1");
-    Key k = new Key(new Text("foo"), StatusSection.NAME, new Text(tableId.canonical()));
-    assertEquals(tableId, StatusSection.getTableId(k));
-  }
-
-  @Test
-  public void failOnNullKeyForTableIdExtract() {
-    Text extractedFile = new Text();
-    assertThrows(NullPointerException.class, () -> StatusSection.getFile(null, extractedFile));
-  }
-
-  @Test
-  public void failOnNullBufferForTableIdExtract() {
-    String file = "hdfs://foo:8020/bar";
-    Key k = new Key(file);
-    Text extractedFile = null;
-    assertThrows(NullPointerException.class, () -> StatusSection.getFile(k, extractedFile));
-  }
-
-  @Test
-  public void failOnIncorrectStatusColfam() {
-    Key k = new Key("file", WorkSection.NAME.toString(), "");
-    assertThrows(IllegalArgumentException.class, () -> StatusSection.getFile(k, new Text()));
-  }
-
-  @Test
-  public void failOnIncorrectWorkColfam() {
-    Key k = new Key("file", StatusSection.NAME.toString(), "");
-    assertThrows(IllegalArgumentException.class, () -> WorkSection.getFile(k, new Text()));
-  }
-
-  @Test
-  public void orderSerialization() {
-    long now = System.currentTimeMillis();
-    Mutation m = OrderSection.createMutation("/accumulo/file", now);
-    Key k = new Key(new Text(m.getRow()));
-    assertEquals("/accumulo/file", OrderSection.getFile(k));
-    assertEquals(now, OrderSection.getTimeClosed(k));
-  }
-
-  @Test
-  public void orderSerializationWithBuffer() {
-    Text buff = new Text();
-    long now = System.currentTimeMillis();
-    Mutation m = OrderSection.createMutation("/accumulo/file", now);
-    Key k = new Key(new Text(m.getRow()));
-    assertEquals("/accumulo/file", OrderSection.getFile(k, buff));
-    assertEquals(now, OrderSection.getTimeClosed(k, buff));
-  }
-
-  @Test
-  public void separatorDoesntInterferWithSplit() {
-    Text buff = new Text();
-    // Cycle through 2*128 values
-    for (long i = 1; i < 258; i++) {
-      Mutation m = OrderSection.createMutation("/accumulo/file", i);
-      Key k = new Key(new Text(m.getRow()));
-      assertEquals("/accumulo/file", OrderSection.getFile(k, buff));
-      assertEquals(i, OrderSection.getTimeClosed(k, buff));
-    }
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.java
deleted file mode 100644
index 9da404e..0000000
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationTargetTest.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.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-
-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.junit.jupiter.api.Test;
-
-@Deprecated
-public class ReplicationTargetTest {
-
-  @Test
-  public void properEquality() {
-    ReplicationTarget expected1 = new ReplicationTarget("foo", "bar", TableId.of("1"));
-
-    assertEquals(expected1, new ReplicationTarget("foo", "bar", TableId.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget("foo", "foo", TableId.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget("bar", "bar", TableId.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget(null, "bar", TableId.of("1")));
-    assertNotEquals(expected1, new ReplicationTarget("foo", null, TableId.of("1")));
-  }
-
-  @Test
-  public void writableOut() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    expected.write(buffer);
-
-    DataInputBuffer input = new DataInputBuffer();
-    input.reset(buffer.getData(), buffer.getLength());
-    ReplicationTarget actual = new ReplicationTarget();
-    actual.readFields(input);
-  }
-
-  @Test
-  public void writableOutWithNulls() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget(null, null, null);
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    expected.write(buffer);
-
-    DataInputBuffer input = new DataInputBuffer();
-    input.reset(buffer.getData(), buffer.getLength());
-    ReplicationTarget actual = new ReplicationTarget();
-    actual.readFields(input);
-  }
-
-  @Test
-  public void staticFromTextHelper() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    expected.write(buffer);
-    Text t = new Text();
-    t.set(buffer.getData(), 0, buffer.getLength());
-
-    assertEquals(expected, ReplicationTarget.from(t));
-  }
-
-  @Test
-  public void staticToTextHelper() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    expected.write(buffer);
-    Text t = new Text();
-    t.set(buffer.getData(), 0, buffer.getLength());
-
-    assertEquals(t, expected.toText());
-  }
-
-  @Test
-  public void staticFromStringHelper() throws Exception {
-    ReplicationTarget expected = new ReplicationTarget("foo", "bar", TableId.of("1"));
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    expected.write(buffer);
-    Text t = new Text();
-    t.set(buffer.getData(), 0, buffer.getLength());
-
-    assertEquals(expected, ReplicationTarget.from(t.toString()));
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java
index c548a09..765778b 100644
--- a/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/AuthenticationTokenTest.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.security;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-import java.security.SecureRandom;
 import java.util.stream.IntStream;
 
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -32,14 +32,12 @@
 
 public class AuthenticationTokenTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
   @Test
   public void testSerializeDeserializeToken() {
     byte[] randomBytes = new byte[12];
     do {
       // random fill, but avoid all zeros case
-      random.nextBytes(randomBytes);
+      RANDOM.get().nextBytes(randomBytes);
     } while (IntStream.range(0, randomBytes.length).allMatch(i -> randomBytes[i] == 0));
 
     byte[] serialized = AuthenticationTokenSerializer.serialize(new PasswordToken(randomBytes));
diff --git a/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java b/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java
index 83d087a..9c7d1dd 100644
--- a/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/singletons/SingletonManagerTest.java
@@ -20,7 +20,6 @@
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
@@ -94,55 +93,6 @@
   }
 
   @Test
-  public void testConnectorPreventsDisable() {
-
-    SingletonManager.setMode(Mode.CONNECTOR);
-    assertEquals(Mode.CONNECTOR, SingletonManager.getMode());
-
-    SingletonReservation resv1 = SingletonManager.getClientReservation();
-
-    assertEquals(1, SingletonManager.getReservationCount());
-
-    SingletonReservation resv2 = SingletonManager.getClientReservation();
-
-    assertEquals(2, SingletonManager.getReservationCount());
-
-    resv1.close();
-    resv2.close();
-
-    assertEquals(0, SingletonManager.getReservationCount());
-
-    assertEquals(new TestService(true, 0, 0), service1);
-    assertEquals(new TestService(true, 1, 0), service2);
-
-    SingletonManager.setMode(Mode.CLIENT);
-    assertEquals(Mode.CLIENT, SingletonManager.getMode());
-
-    assertEquals(new TestService(false, 0, 1), service1);
-    assertEquals(new TestService(false, 1, 1), service2);
-
-    assertThrows(IllegalStateException.class, () -> SingletonManager.setMode(Mode.CONNECTOR),
-        "Should only be able to set mode to CONNECTOR once");
-
-    assertEquals(Mode.CLIENT, SingletonManager.getMode());
-  }
-
-  @Test
-  public void testConnectorEnables() {
-    SingletonReservation resv1 = SingletonManager.getClientReservation();
-    resv1.close();
-
-    assertEquals(new TestService(false, 0, 1), service1);
-    assertEquals(new TestService(false, 1, 1), service2);
-
-    // this should enable services
-    SingletonManager.setMode(Mode.CONNECTOR);
-
-    assertEquals(new TestService(true, 1, 1), service1);
-    assertEquals(new TestService(true, 2, 1), service2);
-  }
-
-  @Test
   public void testServerPreventsDisable() {
 
     SingletonManager.setMode(Mode.SERVER);
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java
index c9c478a..e003500 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -35,7 +35,7 @@
 import org.apache.accumulo.core.manager.balancer.TableStatisticsImpl;
 import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TableStatistics;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
@@ -127,35 +127,35 @@
     servers.put("192.168.0.15", "r03s05");
 
     allTabletServers.put(new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.2", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.3", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.4", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.5", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.6", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.7", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.8", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.9", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.10", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.11", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.12", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.13", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.14", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
     allTabletServers.put(new TabletServerIdImpl("192.168.0.15", 9997, Integer.toHexString(1)),
-        new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+        new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
 
     initialTableLocation.put(FOO.getTableName(),
         new TabletServerIdImpl("192.168.0.1", 9997, Integer.toHexString(1)));
@@ -249,7 +249,7 @@
     TreeMap<TabletServerId,TServerStatus> current = new TreeMap<>();
     for (int i = 1; i <= numTservers; i++) {
       TServerStatusImpl status =
-          new TServerStatusImpl(new org.apache.accumulo.core.master.thrift.TabletServerStatus());
+          new TServerStatusImpl(new org.apache.accumulo.core.manager.thrift.TabletServerStatus());
       Map<String,TableStatistics> tableMap = new HashMap<>();
       tableMap.put(FOO.getId().canonical(), new TableStatisticsImpl(new TableInfo()));
       tableMap.put(BAR.getId().canonical(), new TableStatisticsImpl(new TableInfo()));
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java
index 3f85ed3..3dfd120 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/GroupBalancerTest.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.spi.balancer;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -49,8 +49,6 @@
 
 public class GroupBalancerTest {
 
-  private static final SecureRandom random = new SecureRandom();
-
   private static final Function<TabletId,String> partitioner =
       input -> (input == null || input.getEndRow() == null) ? null
           : input.getEndRow().toString().substring(0, 2);
@@ -118,7 +116,7 @@
 
         for (TabletServerId tsi : tservers) {
           current.put(tsi, new TServerStatusImpl(
-              new org.apache.accumulo.core.master.thrift.TabletServerStatus()));
+              new org.apache.accumulo.core.manager.thrift.TabletServerStatus()));
         }
 
         balancer.balance(new BalanceParamsImpl(current, migrations, migrationsOut));
@@ -326,8 +324,8 @@
 
     for (int g = 1; g <= 60; g++) {
       for (int t = 1; t <= 241; t++) {
-        tservers.addTablet(String.format("%02d:%d", g, t), "192.168.1." + (random.nextInt(249) + 1),
-            9997);
+        tservers.addTablet(String.format("%02d:%d", g, t),
+            "192.168.1." + (RANDOM.get().nextInt(249) + 1), 9997);
       }
     }
 
@@ -343,9 +341,9 @@
     TabletServers tservers = new TabletServers();
 
     for (int g = 1; g <= 60; g++) {
-      for (int t = 1; t <= random.nextInt(1000); t++) {
-        tservers.addTablet(String.format("%02d:%d", g, t), "192.168.1." + (random.nextInt(249) + 1),
-            9997);
+      for (int t = 1; t <= RANDOM.get().nextInt(1000); t++) {
+        tservers.addTablet(String.format("%02d:%d", g, t),
+            "192.168.1." + (RANDOM.get().nextInt(249) + 1), 9997);
       }
     }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java
index 53889be..9b7c2b7 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/SimpleLoadBalancerTest.java
@@ -41,7 +41,7 @@
 import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
 import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
@@ -57,8 +57,8 @@
     List<TabletId> tablets = new ArrayList<>();
 
     TServerStatus getStatus() {
-      org.apache.accumulo.core.master.thrift.TabletServerStatus result =
-          new org.apache.accumulo.core.master.thrift.TabletServerStatus();
+      org.apache.accumulo.core.manager.thrift.TabletServerStatus result =
+          new org.apache.accumulo.core.manager.thrift.TabletServerStatus();
       result.tableMap = new HashMap<>();
       for (TabletId tabletId : tablets) {
         TableInfo info = result.tableMap.get(tabletId.getTable().canonical());
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java
index 9d856e6..41af948 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/balancer/TableLoadBalancerTest.java
@@ -43,7 +43,7 @@
 import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
 import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
@@ -63,8 +63,8 @@
   }
 
   private static TServerStatus status(Object... config) {
-    org.apache.accumulo.core.master.thrift.TabletServerStatus thriftStatus =
-        new org.apache.accumulo.core.master.thrift.TabletServerStatus();
+    org.apache.accumulo.core.manager.thrift.TabletServerStatus thriftStatus =
+        new org.apache.accumulo.core.manager.thrift.TabletServerStatus();
     thriftStatus.tableMap = new HashMap<>();
     String tablename = null;
     for (Object c : config) {
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
index 8423ee8..9f4f9d3 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
@@ -20,6 +20,7 @@
 
 import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -45,18 +46,17 @@
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
 import org.apache.accumulo.core.spi.compaction.CompactionPlan.Builder;
+import org.apache.accumulo.core.spi.compaction.CompactionPlanner.InitParameters;
 import org.apache.accumulo.core.util.ConfigurationImpl;
 import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
 import org.apache.accumulo.core.util.compaction.CompactionJobImpl;
 import org.apache.accumulo.core.util.compaction.CompactionPlanImpl;
 import org.apache.accumulo.core.util.compaction.CompactionPlannerInitParams;
-import org.apache.accumulo.core.util.compaction.CompactionServicesConfig;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.gson.JsonParseException;
 
 public class DefaultCompactionPlannerTest {
 
@@ -67,11 +67,10 @@
   private static final Configuration defaultConf =
       new ConfigurationImpl(DefaultConfiguration.getInstance());
   private static final CompactionServiceId csid = CompactionServiceId.of("cs1");
-
-  private static final Logger log = LoggerFactory.getLogger(DefaultCompactionPlannerTest.class);
+  private static final String prefix = Property.COMPACTION_SERVICE_PREFIX.getKey();
 
   @Test
-  public void testFindFilesToCompact() {
+  public void testFindFilesToCompact() throws Exception {
 
     testFFtC(createCFs("F4", "1M", "F5", "1M", "F6", "1M"),
         createCFs("F1", "100M", "F2", "100M", "F3", "100M", "F4", "1M", "F5", "1M", "F6", "1M"),
@@ -149,7 +148,7 @@
   }
 
   @Test
-  public void testRunningCompaction() {
+  public void testRunningCompaction() throws Exception {
     String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
         + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2},"
         + "{'name':'large','type': 'internal','maxSize':'512M','numThreads':3},"
@@ -181,127 +180,10 @@
     assertEquals(CompactionExecutorIdImpl.internalId(csid, "medium"), job.getExecutor());
   }
 
-  /**
-   * Tests that the maxOpen property overrides the deprecated open.max property with the default
-   * service
-   */
   @Test
-  @SuppressWarnings("removal")
-  public void testOverrideMaxOpenDefaultService() {
-    Map<String,String> overrides = new HashMap<>();
-    // Set old property and use that for max open files.
-    overrides.put(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), "17");
-    SiteConfiguration aconf = SiteConfiguration.empty().withOverrides(overrides).build();
-    ConfigurationImpl config = new ConfigurationImpl(aconf);
-
-    ServiceEnvironment senv = EasyMock.createMock(ServiceEnvironment.class);
-    EasyMock.expect(senv.getConfiguration()).andReturn(config).anyTimes();
-    EasyMock.replay(senv);
-
-    // Use the CompactionServicesConfig to create options based on default property values
-    var compactionServices = new CompactionServicesConfig(aconf, log::warn);
-    var options = compactionServices.getOptions().get("default");
-
-    var initParams =
-        new CompactionPlannerInitParams(CompactionServiceId.of("default"), options, senv);
-
-    var planner = new DefaultCompactionPlanner();
-    planner.init(initParams);
-
-    var all = createCFs("F1", "10M", "F2", "11M", "F3", "12M", "F4", "13M", "F5", "14M", "F6",
-        "15M", "F7", "16M", "F8", "17M", "F9", "18M", "FA", "19M", "FB", "20M", "FC", "21M", "FD",
-        "22M", "FE", "23M", "FF", "24M", "FG", "25M", "FH", "26M");
-    Set<CompactionJob> compacting = Set.of();
-    var params = createPlanningParams(all, all, compacting, 2, CompactionKind.USER);
-    var plan = planner.makePlan(params);
-    var job = getOnlyElement(plan.getJobs());
-    assertEquals(all, job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.internalId(CompactionServiceId.of("default"), "large"),
-        job.getExecutor());
-
-    overrides.put(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), "5");
-    aconf = SiteConfiguration.empty().withOverrides(overrides).build();
-    config = new ConfigurationImpl(aconf);
-    senv = EasyMock.createMock(ServiceEnvironment.class);
-    EasyMock.expect(senv.getConfiguration()).andReturn(config).anyTimes();
-    EasyMock.replay(senv);
-
-    // Create new initParams so executor IDs can be reused
-    initParams = new CompactionPlannerInitParams(CompactionServiceId.of("default"), options, senv);
-    planner = new DefaultCompactionPlanner();
-    planner.init(initParams);
-
-    params = createPlanningParams(all, all, compacting, 2, CompactionKind.USER);
-    plan = planner.makePlan(params);
-    job = getOnlyElement(plan.getJobs());
-    assertEquals(createCFs("F1", "10M", "F2", "11M", "F3", "12M", "F4", "13M", "F5", "14M"),
-        job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.internalId(CompactionServiceId.of("default"), "medium"),
-        job.getExecutor());
-  }
-
-  /**
-   * Tests that the maxOpen property overrides the deprecated open.max property
-   */
-  @Test
-  @SuppressWarnings("removal")
-  public void testOverrideMaxOpen() {
-    Map<String,String> overrides = new HashMap<>();
-    // Set old property and use that for max open files.
-    overrides.put(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), "17");
-    SiteConfiguration aconf = SiteConfiguration.empty().withOverrides(overrides).build();
-    ConfigurationImpl config = new ConfigurationImpl(aconf);
-
-    String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
-        + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2},"
-        + "{'name':'large','type': 'internal','maxSize':'512M','numThreads':3},"
-        + "{'name':'huge','type': 'internal','numThreads':4}]";
-
-    var planner = createPlanner(config, executors);
-    var all = createCFs("F1", "1M", "F2", "2M", "F3", "4M", "F4", "8M", "F5", "16M", "F6", "32M",
-        "F7", "64M", "F8", "128M", "F9", "256M", "FA", "512M", "FB", "1G", "FC", "2G", "FD", "4G",
-        "FE", "8G", "FF", "16G", "FG", "32G", "FH", "64G");
-    Set<CompactionJob> compacting = Set.of();
-    var params = createPlanningParams(all, all, compacting, 2, CompactionKind.USER);
-    var plan = planner.makePlan(params);
-    var job = getOnlyElement(plan.getJobs());
-    assertEquals(all, job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.internalId(csid, "huge"), job.getExecutor());
-
-    // Set new property that overrides the old property.
-    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen",
-        "15");
-    aconf = SiteConfiguration.empty().withOverrides(overrides).build();
-    config = new ConfigurationImpl(aconf);
-    planner = createPlanner(config, executors);
-    params = createPlanningParams(all, all, compacting, 2, CompactionKind.USER);
-    plan = planner.makePlan(params);
-
-    // 17 files that do not meet the compaction ratio. When max files to compact is 15,
-    // the plan should do 3 files then 15
-    job = getOnlyElement(plan.getJobs());
-    assertEquals(createCFs("F1", "1M", "F2", "2M", "F3", "4M"), job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.internalId(csid, "small"), job.getExecutor());
-
-    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen",
-        "5");
-    aconf = SiteConfiguration.empty().withOverrides(overrides).build();
-    // 17 files that do not meet the compaction ratio. When max files to compact is 5 should do 5,
-    // files then another 5, then the final 5.
-    config = new ConfigurationImpl(aconf);
-    planner = createPlanner(config, executors);
-    params = createPlanningParams(all, all, compacting, 2, CompactionKind.USER);
-    plan = planner.makePlan(params);
-    job = getOnlyElement(plan.getJobs());
-    assertEquals(createCFs("F4", "8M", "F3", "4M", "F2", "2M", "F1", "1M", "F5", "16M"),
-        job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.internalId(csid, "small"), job.getExecutor());
-  }
-
-  @Test
-  public void testUserCompaction() {
+  public void testUserCompaction() throws Exception {
     ConfigurationCopy aconf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    aconf.set(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen", "15");
+    aconf.set(prefix + "cs1.planner.opts.maxOpen", "15");
     ConfigurationImpl config = new ConfigurationImpl(aconf);
 
     String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
@@ -372,7 +254,7 @@
   }
 
   @Test
-  public void testMaxSize() {
+  public void testMaxSize() throws Exception {
     String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
         + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2},"
         + "{'name':'large','type': 'internal','maxSize':'512M','numThreads':3}]";
@@ -395,6 +277,61 @@
     assertEquals(CompactionExecutorIdImpl.internalId(csid, "large"), job.getExecutor());
   }
 
+  @Test
+  public void testQueueCreation() throws Exception {
+    DefaultCompactionPlanner planner = new DefaultCompactionPlanner();
+
+    String queues = "[{\"name\": \"small\", \"maxSize\":\"32M\"},{\"name\":\"midsize\"}]";
+    planner.init(getInitParamQueues(defaultConf, queues));
+
+    var all = createCFs("F1", "1M", "F2", "1M", "F3", "1M", "F4", "1M");
+    var params = createPlanningParams(all, all, Set.of(), 2, CompactionKind.SYSTEM);
+    var plan = planner.makePlan(params);
+
+    var job = getOnlyElement(plan.getJobs());
+    assertEquals(all, job.getFiles());
+    assertEquals(CompactionExecutorIdImpl.externalId("small"), job.getExecutor());
+
+    all = createCFs("F1", "100M", "F2", "100M", "F3", "100M", "F4", "100M");
+    params = createPlanningParams(all, all, Set.of(), 2, CompactionKind.SYSTEM);
+    plan = planner.makePlan(params);
+
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(all, job.getFiles());
+    assertEquals(CompactionExecutorIdImpl.externalId("midsize"), job.getExecutor());
+  }
+
+  /**
+   * Tests that additional fields in the JSON objects cause errors to be thrown.
+   */
+  @Test
+  public void testErrorAdditionalConfigFields() {
+    DefaultCompactionPlanner QueuePlanner = new DefaultCompactionPlanner();
+
+    String queues =
+        "[{\"name\":\"smallQueue\", \"maxSize\":\"32M\"}, {\"name\":\"largeQueue\", \"type\":\"internal\", \"foo\":\"bar\", \"queue\":\"broken\"}]";
+
+    final InitParameters queueParams = getInitParamQueues(defaultConf, queues);
+    assertNotNull(queueParams);
+    var e = assertThrows(JsonParseException.class, () -> QueuePlanner.init(queueParams),
+        "Failed to throw error");
+    assertTrue(e.getMessage().contains("[type, foo, queue]"),
+        "Error message didn't contain '[type, foo, queue]'");
+
+    String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
+        + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2, 'foo':'bar'},"
+        + "{'name':'large','type': 'internal','numThreads':1, 'unexpectedField':'foo'}]";
+
+    final InitParameters execParams = getInitParams(defaultConf, executors);
+    assertNotNull(execParams);
+
+    DefaultCompactionPlanner ExecPlanner = new DefaultCompactionPlanner();
+    var err = assertThrows(JsonParseException.class, () -> ExecPlanner.init(execParams),
+        "Failed to throw error");
+    assertTrue(err.getMessage().contains("Invalid fields: [foo]"),
+        "Error message didn't contain '[foo]'");
+  }
+
   /**
    * Tests internal type executor with no numThreads set throws error
    */
@@ -441,6 +378,45 @@
   }
 
   /**
+   * Tests queue with missing name throws error
+   */
+  @Test
+  public void testErrorQueueNoName() {
+    DefaultCompactionPlanner planner = new DefaultCompactionPlanner();
+    String queues = "[{\"name\":\"smallQueue\", \"maxSize\":\"32M\"}, {\"maxSize\":\"120M\"}]";
+
+    final InitParameters params = getInitParamQueues(defaultConf, queues);
+    assertNotNull(params);
+
+    var e = assertThrows(NullPointerException.class, () -> planner.init(params),
+        "Failed to throw error");
+    assertEquals(e.getMessage(), "'name' must be specified", "Error message didn't contain 'name'");
+  }
+
+  /**
+   * Tests not having executors or queues throws errors
+   */
+  @Test
+  public void testErrorNoExecutors() {
+    DefaultCompactionPlanner planner = new DefaultCompactionPlanner();
+    var execParams = getInitParams(defaultConf, "");
+    assertNotNull(execParams);
+
+    var e = assertThrows(IllegalStateException.class, () -> planner.init(execParams),
+        "Failed to throw error");
+    assertEquals("No defined executors or queues for this planner", e.getMessage(),
+        "Error message was not equal");
+
+    var params = getInitParamQueues(defaultConf, "");
+    assertNotNull(params);
+
+    var e2 = assertThrows(IllegalStateException.class, () -> planner.init(params),
+        "Failed to throw error");
+    assertEquals("No defined executors or queues for this planner", e2.getMessage(),
+        "Error message was not equal");
+  }
+
+  /**
    * Tests executors can only have one without a max size.
    */
   @Test
@@ -470,52 +446,17 @@
     assertTrue(e.getMessage().contains("maxSize"), "Error message didn't contain maxSize");
   }
 
-  /**
-   * Tests when "executors" is defined but empty.
-   */
-  @Test
-  public void testErrorEmptyExecutors() {
-    DefaultCompactionPlanner planner = new DefaultCompactionPlanner();
-    String executors = "";
-
-    var e = assertThrows(IllegalStateException.class,
-        () -> planner.init(getInitParams(defaultConf, executors)), "Failed to throw error");
-    assertTrue(e.getMessage().contains("No defined executors"),
-        "Error message didn't contain 'No defined executors'");
-  }
-
-  /**
-   * Tests when "executors" doesn't exist
-   */
-  @Test
-  public void testErrorNoExecutors() {
-
-    ServiceEnvironment senv = EasyMock.createMock(ServiceEnvironment.class);
-    EasyMock.expect(senv.getConfiguration()).andReturn(defaultConf).anyTimes();
-    EasyMock.replay(senv);
-
-    var initParams = new CompactionPlannerInitParams(csid, new HashMap<>(), senv);
-
-    DefaultCompactionPlanner dcPlanner = new DefaultCompactionPlanner();
-
-    var e = assertThrows(IllegalStateException.class, () -> dcPlanner.init(initParams),
-        "Failed to throw error");
-    assertTrue(e.getMessage().contains("No defined executors"),
-        "Error message didn't contain 'No defined executors'");
-  }
-
   // Test cases where a tablet has more than table.file.max files, but no files were found using the
   // compaction ratio. The planner should try to find the highest ratio that will result in a
   // compaction.
   @Test
-  public void testMaxTabletFiles() {
+  public void testMaxTabletFiles() throws Exception {
     String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
         + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2},"
         + "{'name':'large','type': 'internal','numThreads':3}]";
 
     Map<String,String> overrides = new HashMap<>();
-    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen",
-        "10");
+    overrides.put(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen", "10");
     overrides.put(Property.TABLE_FILE_MAX.getKey(), "7");
     var conf = new ConfigurationImpl(SiteConfiguration.empty().withOverrides(overrides).build());
 
@@ -586,14 +527,13 @@
   }
 
   @Test
-  public void testMaxTabletFilesNoCompaction() {
+  public void testMaxTabletFilesNoCompaction() throws Exception {
     String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
         + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2},"
         + "{'name':'large','type': 'internal','maxSize':'512M','numThreads':3}]";
 
     Map<String,String> overrides = new HashMap<>();
-    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen",
-        "10");
+    overrides.put(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen", "10");
     overrides.put(Property.TABLE_FILE_MAX.getKey(), "7");
     var conf = new ConfigurationImpl(SiteConfiguration.empty().withOverrides(overrides).build());
 
@@ -625,14 +565,13 @@
 
   // Test to ensure that plugin falls back from TABLE_FILE_MAX to TSERV_SCAN_MAX_OPENFILES
   @Test
-  public void testMaxTableFilesFallback() {
+  public void testMaxTableFilesFallback() throws Exception {
     String executors = "[{'name':'small','type': 'internal','maxSize':'32M','numThreads':1},"
         + "{'name':'medium','type': 'internal','maxSize':'128M','numThreads':2},"
         + "{'name':'large','type': 'internal','numThreads':3}]";
 
     Map<String,String> overrides = new HashMap<>();
-    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen",
-        "10");
+    overrides.put(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen", "10");
     overrides.put(Property.TABLE_FILE_MAX.getKey(), "0");
     overrides.put(Property.TSERV_SCAN_MAX_OPENFILES.getKey(), "5");
     var conf = new ConfigurationImpl(SiteConfiguration.empty().withOverrides(overrides).build());
@@ -653,7 +592,8 @@
   }
 
   // Create a set of files whose sizes would require certain compaction ratios to compact
-  private Set<CompactableFile> createCFs(int initialSize, double... desiredRatios) {
+  private Set<CompactableFile> createCFs(int initialSize, double... desiredRatios)
+      throws URISyntaxException {
     List<String> pairs = new ArrayList<>();
     pairs.add("F1");
     pairs.add(initialSize + "");
@@ -689,18 +629,15 @@
     return createCFs(pairs.toArray(new String[0]));
   }
 
-  private static Set<CompactableFile> createCFs(String... namesSizePairs) {
+  private static Set<CompactableFile> createCFs(String... namesSizePairs)
+      throws URISyntaxException {
     Set<CompactableFile> files = new HashSet<>();
 
     for (int i = 0; i < namesSizePairs.length; i += 2) {
       String name = namesSizePairs[i];
       long size = ConfigurationTypeHelper.getFixedMemoryAsBytes(namesSizePairs[i + 1]);
-      try {
-        files.add(CompactableFile
-            .create(new URI("hdfs://fake/accumulo/tables/1/t-0000000z/" + name + ".rf"), size, 0));
-      } catch (URISyntaxException e) {
-        throw new RuntimeException(e);
-      }
+      files.add(CompactableFile
+          .create(new URI("hdfs://fake/accumulo/tables/1/t-0000000z/" + name + ".rf"), size, 0));
     }
 
     return files;
@@ -792,11 +729,28 @@
     };
   }
 
+  private static CompactionPlanner.InitParameters getInitParamQueues(Configuration conf,
+      String queues) {
+
+    String maxOpen = conf.get(prefix + "cs1.planner.opts.maxOpen");
+    Map<String,String> options = new HashMap<>();
+    options.put("queues", queues.replaceAll("'", "\""));
+
+    if (maxOpen != null) {
+      options.put("maxOpen", maxOpen);
+    }
+
+    ServiceEnvironment senv = EasyMock.createMock(ServiceEnvironment.class);
+    EasyMock.expect(senv.getConfiguration()).andReturn(conf).anyTimes();
+    EasyMock.replay(senv);
+
+    return new CompactionPlannerInitParams(csid, prefix, options, senv);
+  }
+
   private static CompactionPlanner.InitParameters getInitParams(Configuration conf,
       String executors) {
 
-    String maxOpen =
-        conf.get(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.maxOpen");
+    String maxOpen = conf.get(prefix + "cs1.planner.opts.maxOpen");
     Map<String,String> options = new HashMap<>();
     options.put("executors", executors.replaceAll("'", "\""));
 
@@ -808,7 +762,7 @@
     EasyMock.expect(senv.getConfiguration()).andReturn(conf).anyTimes();
     EasyMock.replay(senv);
 
-    return new CompactionPlannerInitParams(csid, options, senv);
+    return new CompactionPlannerInitParams(csid, prefix, options, senv);
   }
 
   private static DefaultCompactionPlanner createPlanner(Configuration conf, String executors) {
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java b/core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java
index abdc3b7..663efce 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java
@@ -31,8 +31,6 @@
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-import com.google.common.util.concurrent.UncheckedExecutionException;
-
 public class SpaceAwareVolumeChooserTest {
 
   VolumeChooserEnvironment chooserEnv = null;
@@ -124,7 +122,7 @@
   @Test
   public void testNoFreeSpace() {
     testSpecificSetup(0L, 0L, null, 1, false);
-    assertThrows(UncheckedExecutionException.class, this::makeChoices);
+    assertThrows(IllegalStateException.class, this::makeChoices);
   }
 
   @Test
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
index 376cfba..59bfd40 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/ConfigurableScanServerSelectorTest.java
@@ -18,12 +18,12 @@
  */
 package org.apache.accumulo.core.spi.scan;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.Collection;
 import java.util.HashMap;
@@ -253,12 +253,11 @@
 
     Map<String,Long> allServersSeen = new HashMap<>();
 
-    SecureRandom rand = new SecureRandom();
-
     for (int t = 0; t < 10000; t++) {
       Set<String> serversSeen = new HashSet<>();
 
-      String endRow = Long.toString(Math.abs(Math.max(rand.nextLong(), Long.MIN_VALUE + 1)), 36);
+      String endRow =
+          Long.toString(Math.abs(Math.max(RANDOM.get().nextLong(), Long.MIN_VALUE + 1)), 36);
 
       var tabletId = t % 1000 == 0 ? nti("" + t, null) : nti("" + t, endRow);
 
diff --git a/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java b/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
index e7aa3ff..579571d 100644
--- a/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/tabletserver/log/LogEntryTest.java
@@ -18,98 +18,107 @@
  */
 package org.apache.accumulo.core.tabletserver.log;
 
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Map.Entry;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.UUID;
+import java.util.stream.Stream;
 
 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.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class LogEntryTest {
 
-  @SuppressWarnings("removal")
-  private static void compareLogEntries(LogEntry one, LogEntry two) throws IOException {
+  private final String validHost = "localhost+9997";
+  private final UUID validUUID = UUID.randomUUID();
+  private final String validPath = "viewfs:/a/accumulo/wal/" + validHost + "/" + validUUID;
+
+  @Test
+  public void testColumnFamily() {
+    assertEquals(new Text("log"), LogColumnFamily.NAME);
+  }
+
+  @Test
+  public void testFromPath() {
+    var logEntry = LogEntry.fromPath(validPath);
+    verifyLogEntry(logEntry, new Text("-/" + validPath));
+  }
+
+  @Test
+  public void testFromMetadata() {
+    var columnQualifier = "prefix/" + validPath;
+    var logEntry = LogEntry.fromMetaWalEntry(
+        new SimpleImmutableEntry<>(new Key("1<", LogColumnFamily.STR_NAME, columnQualifier), null));
+    verifyLogEntry(logEntry, new Text(columnQualifier));
+  }
+
+  // helper for testing build from constructor or from metadata
+  private void verifyLogEntry(LogEntry logEntry, Text expectedColumnQualifier) {
+    assertEquals(validPath, logEntry.toString());
+    assertEquals(validPath, logEntry.getPath());
+    assertEquals(HostAndPort.fromString(validHost.replace('+', ':')), logEntry.getTServer());
+    assertEquals(expectedColumnQualifier, logEntry.getColumnQualifier());
+    assertEquals(validUUID, logEntry.getUniqueID());
+  }
+
+  @Test
+  public void testEquals() {
+    LogEntry one = LogEntry.fromPath(validPath);
+    LogEntry two = LogEntry.fromPath(validPath);
+
     assertNotSame(one, two);
     assertEquals(one.toString(), two.toString());
-    assertEquals(one.getColumnFamily(), two.getColumnFamily());
+    assertEquals(one.getPath(), two.getPath());
+    assertEquals(one.getTServer(), two.getTServer());
     assertEquals(one.getColumnQualifier(), two.getColumnQualifier());
-    assertEquals(one.getRow(), two.getRow());
     assertEquals(one.getUniqueID(), two.getUniqueID());
-    assertEquals(one.getValue(), two.getValue());
-    // arrays differ in serialized form because of the different prevEndRow, but that shouldn't
-    // matter for anything functionality in the LogEntry
-    assertFalse(Arrays.equals(one.toBytes(), two.toBytes()));
+    assertEquals(one, two);
+
+    assertEquals(one, one);
+    assertEquals(two, two);
   }
 
   @Test
-  public void testPrevRowDoesntMatter() throws IOException {
-    long ts = 12345678L;
-    String filename = "default/foo";
+  public void testValidPaths() {
+    var validPath1 = validHost + "/" + validUUID;
+    var validPath2 = "dir1/" + validPath1;
+    var validPath3 = "dir2/" + validPath2;
 
-    // with no end row, different prev rows
-    LogEntry entry1 =
-        new LogEntry(new KeyExtent(TableId.of("1"), null, new Text("A")), ts, filename);
-    LogEntry entry2 =
-        new LogEntry(new KeyExtent(TableId.of("1"), null, new Text("B")), ts, filename);
-    assertEquals("1< default/foo", entry1.toString());
-    compareLogEntries(entry1, entry2);
-
-    // with same end row, different prev rows
-    LogEntry entry3 =
-        new LogEntry(new KeyExtent(TableId.of("2"), new Text("same"), new Text("A")), ts, filename);
-    LogEntry entry4 =
-        new LogEntry(new KeyExtent(TableId.of("2"), new Text("same"), new Text("B")), ts, filename);
-    assertEquals("2;same default/foo", entry3.toString());
-    compareLogEntries(entry3, entry4);
+    Stream.of(validPath1, validPath2, validPath3)
+        .forEach(s -> assertDoesNotThrow(() -> LogEntry.fromPath(s)));
   }
 
   @Test
-  public void test() throws Exception {
-    KeyExtent extent = new KeyExtent(TableId.of("1"), null, null);
-    long ts = 12345678L;
-    String filename = "default/foo";
-    LogEntry entry = new LogEntry(extent, ts, filename);
-    assertEquals(extent.toMetaRow(), entry.getRow());
-    assertEquals(filename, entry.filename);
-    assertEquals(ts, entry.timestamp);
-    assertEquals("1< default/foo", entry.toString());
-    assertEquals(new Text("log"), entry.getColumnFamily());
-    assertEquals(new Text("-/default/foo"), entry.getColumnQualifier());
-    @SuppressWarnings("removal")
-    LogEntry copy = LogEntry.fromBytes(entry.toBytes());
-    assertEquals(entry.toString(), copy.toString());
-    Key key = new Key(new Text("1<"), new Text("log"), new Text("localhost:1234/default/foo"));
-    key.setTimestamp(ts);
-    var mapEntry = new Entry<Key,Value>() {
-      @Override
-      public Key getKey() {
-        return key;
-      }
+  public void testBadPathLength() {
+    Stream.of("foo", "", validHost).forEach(badPath -> {
+      var e = assertThrows(IllegalArgumentException.class, () -> LogEntry.fromPath(badPath));
+      assertTrue(e.getMessage().contains("The path should end with tserver/UUID."));
+    });
+  }
 
-      @Override
-      public Value getValue() {
-        return entry.getValue();
-      }
+  @Test
+  public void testInvalidHostPort() {
+    Stream.of("default:9997", "default+badPort").forEach(badHostAndPort -> {
+      var badPath = badHostAndPort + "/" + validUUID;
+      var e = assertThrows(IllegalArgumentException.class, () -> LogEntry.fromPath(badPath));
+      assertTrue(e.getMessage().contains("Expected: host+port. Found '" + badHostAndPort + "'"));
+    });
+  }
 
-      @Override
-      public Value setValue(Value value) {
-        throw new UnsupportedOperationException();
-      }
-    };
-    LogEntry copy2 = LogEntry.fromMetaWalEntry(mapEntry);
-    assertEquals(entry.toString(), copy2.toString());
-    assertEquals(entry.timestamp, copy2.timestamp);
-    assertEquals("foo", entry.getUniqueID());
-    assertEquals("-/default/foo", entry.getColumnQualifier().toString());
-    assertEquals(new Value("default/foo"), entry.getValue());
+  @Test
+  public void testInvalidUUID() {
+    var badUUID = "badUUID";
+    var pathWithBadUUID = validHost + "/" + badUUID;
+    var e = assertThrows(IllegalArgumentException.class, () -> LogEntry.fromPath(pathWithBadUUID));
+    assertTrue(e.getMessage().contains("Expected valid UUID. Found '" + badUUID + "'"));
   }
 
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java b/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java
index e81b72d..4c1303a 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/AddressUtilTest.java
@@ -28,6 +28,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Test the AddressUtil class.
  */
@@ -102,4 +104,28 @@
         "The JVM Security settings cache DNS failures forever, this should cause an exception.");
 
   }
+
+  @Test
+  public void normalizeAddressRequirePortTest() {
+    HostAndPort hostAndPort = AddressUtil.parseAddress("127.0.1.2+8080");
+    assertEquals("127.0.1.2", hostAndPort.getHost());
+    assertEquals(8080, hostAndPort.getPort());
+
+    HostAndPort hostAndPort2 = AddressUtil.parseAddress("127.0.1.2:9123");
+    assertEquals("127.0.1.2", hostAndPort2.getHost());
+    assertEquals(9123, hostAndPort2.getPort());
+
+    assertThrows(IllegalArgumentException.class, () -> AddressUtil.parseAddress("127.0.1.2"));
+  }
+
+  @Test
+  public void normalizeAddressWithDefaultTest() {
+    HostAndPort hostAndPort = AddressUtil.parseAddress("127.0.1.2+8080", 9123);
+    assertEquals("127.0.1.2", hostAndPort.getHost());
+    assertEquals(8080, hostAndPort.getPort());
+
+    HostAndPort hostAndPort2 = AddressUtil.parseAddress("127.0.1.2", 9123);
+    assertEquals("127.0.1.2", hostAndPort2.getHost());
+    assertEquals(9123, hostAndPort2.getPort());
+  }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java b/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
index 9194882..069b7e0 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/ByteBufferUtilTest.java
@@ -25,6 +25,7 @@
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
@@ -52,7 +53,7 @@
       ByteBufferUtil.write(dos, bb);
       dos.close();
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
 
     assertEquals(expected, new String(baos.toByteArray(), UTF_8));
@@ -63,7 +64,7 @@
       bais.read(buffer);
       assertEquals(expected, new String(buffer, UTF_8));
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/util/HostAndPortTest.java b/core/src/test/java/org/apache/accumulo/core/util/HostAndPortComparatorTest.java
similarity index 77%
rename from core/src/test/java/org/apache/accumulo/core/util/HostAndPortTest.java
rename to core/src/test/java/org/apache/accumulo/core/util/HostAndPortComparatorTest.java
index 2a4f34f..fae4dbb 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/HostAndPortTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/HostAndPortComparatorTest.java
@@ -21,6 +21,7 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
+import java.util.Comparator;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
@@ -29,57 +30,62 @@
 
 import org.junit.jupiter.api.Test;
 
-class HostAndPortTest {
+import com.google.common.net.HostAndPort;
+
+class HostAndPortComparatorTest {
+
+  final static private Comparator<HostAndPort> COMPARATOR = new HostAndPortComparator();
 
   @Test
-  void testCompareTo() {
+  void testCompare() {
+
     HostAndPort hostAndPort1 = HostAndPort.fromString("example.info");
     HostAndPort hostAndPort2 = HostAndPort.fromString("example.com");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) > 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) > 0);
 
     HostAndPort hostPortSame = HostAndPort.fromString("www.test.com");
-    assertTrue(hostPortSame.compareTo(hostPortSame) == 0);
+    assertTrue(COMPARATOR.compare(hostPortSame, hostPortSame) == 0);
 
     hostAndPort1 = HostAndPort.fromString("www.example.com");
     hostAndPort2 = HostAndPort.fromString("www.example.com");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) == 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) == 0);
 
     hostAndPort1 = HostAndPort.fromString("192.0.2.1:80");
     hostAndPort2 = HostAndPort.fromString("192.0.2.1");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) > 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) > 0);
 
     hostAndPort1 = HostAndPort.fromString("[2001:db8::1]");
     hostAndPort2 = HostAndPort.fromString("[2001:db9::1]");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) < 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) < 0);
 
     hostAndPort1 = HostAndPort.fromString("2001:db8:3333:4444:5555:6676:7777:8888");
     hostAndPort2 = HostAndPort.fromString("2001:db8:3333:4444:5555:6666:7777:8888");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) > 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) > 0);
 
     hostAndPort1 = HostAndPort.fromString("192.0.2.1:80");
     hostAndPort2 = HostAndPort.fromString("192.1.2.1");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) < 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) < 0);
 
     hostAndPort1 = HostAndPort.fromString("12.1.2.1");
     hostAndPort2 = HostAndPort.fromString("192.1.2.1");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) < 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) < 0);
 
     hostAndPort1 = HostAndPort.fromString("wwww.example.com");
     hostAndPort2 = HostAndPort.fromString("192.1.2.1");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) > 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) > 0);
 
     hostAndPort1 = HostAndPort.fromString("2001:db8::1");
     hostAndPort2 = HostAndPort.fromString("2001:db9::1");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) < 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) < 0);
 
     hostAndPort1 = HostAndPort.fromString("");
     hostAndPort2 = HostAndPort.fromString("2001:db9::1");
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) < 0);
-    assertTrue(hostAndPort2.compareTo(hostAndPort1) > 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) < 0);
+    assertTrue(COMPARATOR.compare(hostAndPort2, hostAndPort1) > 0);
 
     hostAndPort1 = HostAndPort.fromString("2001:db8::1");
     hostAndPort2 = null;
-    assertTrue(hostAndPort1.compareTo(hostAndPort2) > 0);
+    assertTrue(COMPARATOR.compare(hostAndPort1, hostAndPort2) > 0);
   }
 
   @Test
@@ -92,7 +98,8 @@
             "2.2.2.2:10000", "192.12.2.1:79", "1.1.1.1:24", "1.1.1.1", "192.12.2.1:79", "a.b.c.d",
             "1.100.100.100", "2.2.2.2:9999", "a.b.b.d", "www.example.com", "www.alpha.org",
             "a.b.c.d:10", "a.b.b.d:10", "a.b.b.d:11")
-        .map(HostAndPort::fromString).collect(Collectors.toCollection(TreeSet::new));
+        .map(HostAndPort::fromString)
+        .collect(Collectors.toCollection(() -> new TreeSet<>(COMPARATOR)));
     hostPortSet.add(HostAndPort.fromParts("localhost", 1));
     hostPortSet.add(HostAndPort.fromParts("localhost", 000001));
 
diff --git a/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java b/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java
index 6d253fe..ff9bb7e 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/MergeTest.java
@@ -28,6 +28,7 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 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.util.Merge.Size;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
@@ -57,8 +58,34 @@
     protected void message(String format, Object... args) {}
 
     @Override
-    protected Iterator<Size> getSizeIterator(AccumuloClient client, String tablename,
-        final Text start, final Text end) throws MergeException {
+    public void mergomatic(AccumuloClient client, String table, Text start, Text end, long goalSize,
+        boolean force) throws MergeException {
+      if (table.equals(AccumuloTable.METADATA.tableName())) {
+        throw new IllegalArgumentException("cannot merge tablets on the metadata table");
+      }
+
+      List<Size> sizes = new ArrayList<>();
+      long totalSize = 0;
+
+      Iterator<Size> sizeIterator = getSizeIterator(start, end);
+
+      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;
+        }
+      }
+      if (sizes.size() > 1) {
+        mergeMany(client, table, sizes, goalSize, force, true);
+      }
+    }
+
+    protected Iterator<Size> getSizeIterator(final Text start, final Text end) {
       final Iterator<Size> impl = tablets.iterator();
       return new Iterator<>() {
         Size next = skip();
diff --git a/core/src/test/java/org/apache/accumulo/core/util/RetryTest.java b/core/src/test/java/org/apache/accumulo/core/util/RetryTest.java
index ce19333..c05189a 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/RetryTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/RetryTest.java
@@ -18,17 +18,12 @@
  */
 package org.apache.accumulo.core.util;
 
-import static java.util.concurrent.TimeUnit.DAYS;
-import static java.util.concurrent.TimeUnit.HOURS;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.util.concurrent.TimeUnit;
+import java.time.Duration;
 
 import org.apache.accumulo.core.util.Retry.NeedsLogInterval;
 import org.apache.accumulo.core.util.Retry.NeedsMaxWait;
@@ -38,6 +33,7 @@
 import org.apache.accumulo.core.util.Retry.RetryFactory;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Nested;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,24 +41,23 @@
 public class RetryTest {
 
   private Retry retry;
-  private static final long INITIAL_WAIT = 1000;
-  private static final long WAIT_INC = 1000;
+  private static final Duration INITIAL_WAIT = Duration.ofSeconds(1);
+  private static final Duration WAIT_INC = Duration.ofSeconds(1);
   private static final double BACKOFF_FACTOR = 1.0;
   private static final long MAX_RETRIES = 5;
-  private static final long LOG_INTERVAL = 1000;
+  private static final Duration LOG_INTERVAL = Duration.ofSeconds(1);
   private Retry unlimitedRetry;
-  private static final TimeUnit MS = MILLISECONDS;
 
   private static final Logger log = LoggerFactory.getLogger(RetryTest.class);
 
   @BeforeEach
   public void setup() {
-    retry = Retry.builder().maxRetries(MAX_RETRIES).retryAfter(INITIAL_WAIT, MS)
-        .incrementBy(WAIT_INC, MS).maxWait(MAX_RETRIES * WAIT_INC, MS).backOffFactor(BACKOFF_FACTOR)
-        .logInterval(LOG_INTERVAL, MS).createRetry();
-    unlimitedRetry = Retry.builder().infiniteRetries().retryAfter(INITIAL_WAIT, MS)
-        .incrementBy(WAIT_INC, MS).maxWait(MAX_RETRIES * WAIT_INC, MS).backOffFactor(BACKOFF_FACTOR)
-        .logInterval(LOG_INTERVAL, MS).createRetry();
+    retry = Retry.builder().maxRetries(MAX_RETRIES).retryAfter(INITIAL_WAIT).incrementBy(WAIT_INC)
+        .maxWait(WAIT_INC.multipliedBy(MAX_RETRIES)).backOffFactor(BACKOFF_FACTOR)
+        .logInterval(LOG_INTERVAL).createRetry();
+    unlimitedRetry = Retry.builder().infiniteRetries().retryAfter(INITIAL_WAIT)
+        .incrementBy(WAIT_INC).maxWait(WAIT_INC.multipliedBy(MAX_RETRIES))
+        .backOffFactor(BACKOFF_FACTOR).logInterval(LOG_INTERVAL).createRetry();
   }
 
   @Test
@@ -109,15 +104,15 @@
     retry.setMaxRetries(MAX_RETRIES);
     retry.setStartWait(INITIAL_WAIT);
     retry.setWaitIncrement(WAIT_INC);
-    retry.setMaxWait(MAX_RETRIES * 1000);
+    retry.setMaxWait(Duration.ofSeconds(1).multipliedBy(MAX_RETRIES));
     retry.setBackOffFactor(1);
     retry.setDoTimeJitter(false);
 
-    long currentWait = INITIAL_WAIT;
+    Duration currentWait = INITIAL_WAIT;
     for (int i = 1; i <= MAX_RETRIES; i++) {
       retry.sleep(currentWait);
       EasyMock.expectLastCall();
-      currentWait += WAIT_INC;
+      currentWait = currentWait.plus(WAIT_INC);
     }
 
     EasyMock.replay(retry);
@@ -136,9 +131,9 @@
     retry.setMaxRetries(MAX_RETRIES);
     retry.setBackOffFactor(1.5);
     retry.setStartWait(INITIAL_WAIT);
-    long waitIncrement = 0, currentWait = INITIAL_WAIT;
+    Duration waitIncrement, currentWait = INITIAL_WAIT;
     retry.setWaitIncrement(WAIT_INC);
-    retry.setMaxWait(MAX_RETRIES * 128000);
+    retry.setMaxWait(Duration.ofSeconds(128).multipliedBy(MAX_RETRIES));
     retry.setDoTimeJitter(false);
     double backOfFactor = 1.5, originalBackoff = 1.5;
 
@@ -146,8 +141,13 @@
       retry.sleep(currentWait);
       double waitFactor = backOfFactor;
       backOfFactor *= originalBackoff;
-      waitIncrement = (long) (Math.ceil(waitFactor * WAIT_INC));
-      currentWait = Math.min(retry.getMaxWait(), INITIAL_WAIT + waitIncrement);
+      waitIncrement = Duration.ofMillis((long) (Math.ceil(waitFactor * WAIT_INC.toMillis())));
+      Duration tempWait = INITIAL_WAIT.plus(waitIncrement);
+      if (tempWait.compareTo(retry.getMaxWait()) > 0) {
+        currentWait = retry.getMaxWait();
+      } else {
+        currentWait = tempWait;
+      }
       EasyMock.expectLastCall();
     }
 
@@ -168,16 +168,16 @@
     retry.setStartWait(INITIAL_WAIT);
     retry.setWaitIncrement(WAIT_INC);
     // Make the last retry not increment in length
-    retry.setMaxWait((MAX_RETRIES - 1) * 1000);
+    retry.setMaxWait(Duration.ofSeconds(MAX_RETRIES - 1));
     retry.setBackOffFactor(1);
     retry.setDoTimeJitter(false);
 
-    long currentWait = INITIAL_WAIT;
+    Duration currentWait = INITIAL_WAIT;
     for (int i = 1; i <= MAX_RETRIES; i++) {
       retry.sleep(currentWait);
       EasyMock.expectLastCall();
       if (i < MAX_RETRIES - 1) {
-        currentWait += WAIT_INC;
+        currentWait = currentWait.plus(WAIT_INC);
       }
     }
 
@@ -247,94 +247,102 @@
   @Test
   public void testInitialWait() {
     NeedsRetryDelay builder = Retry.builder().maxRetries(10);
-    builder.retryAfter(10, NANOSECONDS);
-    builder.retryAfter(10, MILLISECONDS);
-    builder.retryAfter(10, DAYS);
-    builder.retryAfter(0, NANOSECONDS);
-    builder.retryAfter(0, MILLISECONDS);
-    builder.retryAfter(0, DAYS);
+    builder.retryAfter(Duration.ofNanos(10));
+    builder.retryAfter(Duration.ofMillis(10));
+    builder.retryAfter(Duration.ofDays(10));
+    builder.retryAfter(Duration.ofNanos(0));
+    builder.retryAfter(Duration.ofMillis(0));
+    builder.retryAfter(Duration.ofDays(0));
 
-    assertThrows(IllegalArgumentException.class, () -> builder.retryAfter(-1, NANOSECONDS),
+    assertThrows(IllegalArgumentException.class, () -> builder.retryAfter(Duration.ofNanos(-1)),
         "Should not allow negative wait times");
   }
 
   @Test
   public void testIncrementBy() {
-    NeedsTimeIncrement builder = Retry.builder().maxRetries(10).retryAfter(10, MILLISECONDS);
-    builder.incrementBy(10, DAYS);
-    builder.incrementBy(10, HOURS);
-    builder.incrementBy(10, NANOSECONDS);
-    builder.incrementBy(0, DAYS);
-    builder.incrementBy(0, HOURS);
-    builder.incrementBy(0, NANOSECONDS);
+    NeedsTimeIncrement builder = Retry.builder().maxRetries(10).retryAfter(Duration.ofMillis(10));
+    builder.incrementBy(Duration.ofDays(10));
+    builder.incrementBy(Duration.ofHours(10));
+    builder.incrementBy(Duration.ofNanos(10));
+    builder.incrementBy(Duration.ofDays(0));
+    builder.incrementBy(Duration.ofHours(0));
+    builder.incrementBy(Duration.ofNanos(0));
 
-    assertThrows(IllegalArgumentException.class, () -> builder.incrementBy(-1, NANOSECONDS),
+    assertThrows(IllegalArgumentException.class, () -> builder.incrementBy(Duration.ofNanos(-1)),
         "Should not allow negative increments");
   }
 
   @Test
   public void testMaxWait() {
-    NeedsMaxWait builder =
-        Retry.builder().maxRetries(10).retryAfter(15, MILLISECONDS).incrementBy(10, MILLISECONDS);
-    builder.maxWait(15, MILLISECONDS);
-    builder.maxWait(16, MILLISECONDS);
+    NeedsMaxWait builder = Retry.builder().maxRetries(10).retryAfter(Duration.ofMillis(15))
+        .incrementBy(Duration.ofMillis(10));
+    builder.maxWait(Duration.ofMillis(15));
+    builder.maxWait(Duration.ofMillis(16));
 
-    assertThrows(IllegalArgumentException.class, () -> builder.maxWait(14, MILLISECONDS),
+    assertThrows(IllegalArgumentException.class, () -> builder.maxWait(Duration.ofMillis(14)),
         "Max wait time should be greater than or equal to initial wait time");
   }
 
   @Test
   public void testLogInterval() {
-    NeedsLogInterval builder = Retry.builder().maxRetries(10).retryAfter(15, MILLISECONDS)
-        .incrementBy(10, MILLISECONDS).maxWait(16, MINUTES).backOffFactor(1);
-    builder.logInterval(10, DAYS);
-    builder.logInterval(10, HOURS);
-    builder.logInterval(10, NANOSECONDS);
-    builder.logInterval(0, DAYS);
-    builder.logInterval(0, HOURS);
-    builder.logInterval(0, NANOSECONDS);
+    NeedsLogInterval builder = Retry.builder().maxRetries(10).retryAfter(Duration.ofMillis(15))
+        .incrementBy(Duration.ofMillis(10)).maxWait(Duration.ofMinutes(16)).backOffFactor(1);
+    builder.logInterval(Duration.ofDays(10));
+    builder.logInterval(Duration.ofHours(10));
+    builder.logInterval(Duration.ofNanos(10));
+    builder.logInterval(Duration.ofDays(0));
+    builder.logInterval(Duration.ofHours(0));
+    builder.logInterval(Duration.ofNanos(0));
 
-    assertThrows(IllegalArgumentException.class, () -> builder.logInterval(-1, NANOSECONDS),
+    assertThrows(IllegalArgumentException.class, () -> builder.logInterval(Duration.ofNanos(-1)),
         "Log interval must not be negative");
   }
 
   @Test
   public void properArgumentsInRetry() {
-    long maxRetries = 10, startWait = 50L, maxWait = 5000L, waitIncrement = 500L,
-        logInterval = 10000L;
-    RetryFactory factory = Retry.builder().maxRetries(maxRetries).retryAfter(startWait, MS)
-        .incrementBy(waitIncrement, MS).maxWait(maxWait, MS).backOffFactor(1)
-        .logInterval(logInterval, MS).createFactory();
+    long maxRetries = 10;
+    Duration startWait = Duration.ofMillis(50);
+    Duration maxWait = Duration.ofMillis(5000);
+    Duration waitIncrement = Duration.ofMillis(500);
+    Duration logInterval = Duration.ofMillis(10000);
+
+    RetryFactory factory =
+        Retry.builder().maxRetries(maxRetries).retryAfter(startWait).incrementBy(waitIncrement)
+            .maxWait(maxWait).backOffFactor(1).logInterval(logInterval).createFactory();
     Retry retry = factory.createRetry();
 
     assertEquals(maxRetries, retry.getMaxRetries());
-    assertEquals(startWait, retry.getCurrentWait());
-    assertEquals(maxWait, retry.getMaxWait());
-    assertEquals(waitIncrement, retry.getWaitIncrement());
-    assertEquals(logInterval, retry.getLogInterval());
+    assertEquals(startWait.toMillis(), retry.getCurrentWait().toMillis());
+    assertEquals(maxWait.toMillis(), retry.getMaxWait().toMillis());
+    assertEquals(waitIncrement.toMillis(), retry.getWaitIncrement().toMillis());
+    assertEquals(logInterval.toMillis(), retry.getLogInterval().toMillis());
   }
 
   @Test
   public void properArgumentsInUnlimitedRetry() {
-    long startWait = 50L, maxWait = 5000L, waitIncrement = 500L, logInterval = 10000L;
+    Duration startWait = Duration.ofMillis(50);
+    Duration maxWait = Duration.ofSeconds(5);
+    Duration waitIncrement = Duration.ofMillis(500);
+    Duration logInterval = Duration.ofSeconds(10);
     double waitFactor = 1.0;
-    RetryFactory factory = Retry.builder().infiniteRetries().retryAfter(startWait, MS)
-        .incrementBy(waitIncrement, MS).maxWait(maxWait, MS).backOffFactor(waitFactor)
-        .logInterval(logInterval, MS).createFactory();
+
+    RetryFactory factory =
+        Retry.builder().infiniteRetries().retryAfter(startWait).incrementBy(waitIncrement)
+            .maxWait(maxWait).backOffFactor(waitFactor).logInterval(logInterval).createFactory();
     Retry retry = factory.createRetry();
 
     assertEquals(-1, retry.getMaxRetries());
-    assertEquals(startWait, retry.getCurrentWait());
-    assertEquals(maxWait, retry.getMaxWait());
-    assertEquals(waitIncrement, retry.getWaitIncrement());
-    assertEquals(logInterval, retry.getLogInterval());
+    assertEquals(startWait.toMillis(), retry.getCurrentWait().toMillis());
+    assertEquals(maxWait.toMillis(), retry.getMaxWait().toMillis());
+    assertEquals(waitIncrement.toMillis(), retry.getWaitIncrement().toMillis());
+    assertEquals(logInterval.toMillis(), retry.getLogInterval().toMillis());
   }
 
   @Test
   public void testInfiniteRetryWithBackoff() throws InterruptedException {
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(500, MILLISECONDS).backOffFactor(1.5)
-        .logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMillis(500)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
     for (int i = 0; i < 100; i++) {
       try {
         retry.waitForNextAttempt(log, i + "");
@@ -344,4 +352,101 @@
       }
     }
   }
+
+  @Nested
+  public class MaxRetriesWithinDuration {
+
+    @Test
+    public void noIncrement() {
+      Duration retriesForDuration = Duration.ofSeconds(3);
+      Duration retryAfter = Duration.ofMillis(100);
+      Retry retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration)
+          .retryAfter(retryAfter).incrementBy(Duration.ofMillis(0)).maxWait(Duration.ofMillis(1000))
+          .backOffFactor(1.0).logInterval(Duration.ofMinutes(3)).createRetry();
+
+      // with no increment, the number of retries should be the duration divided by the retryAfter
+      // (which is used as the initial wait and in this case does not change)
+      long expectedRetries = retriesForDuration.dividedBy(retryAfter);
+      assertEquals(expectedRetries, retry.getMaxRetries());
+
+      // try again with lots of expected retries
+      retriesForDuration = Duration.ofSeconds(30);
+      retryAfter = Duration.ofMillis(10);
+      retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration).retryAfter(retryAfter)
+          .incrementBy(Duration.ofMillis(0)).maxWait(Duration.ofMillis(1000)).backOffFactor(1.0)
+          .logInterval(Duration.ofMinutes(3)).createRetry();
+
+      expectedRetries = retriesForDuration.dividedBy(retryAfter);
+      assertEquals(expectedRetries, retry.getMaxRetries());
+    }
+
+    @Test
+    public void withIncrement() {
+      final Duration retriesForDuration = Duration.ofMillis(1500);
+      final Duration retryAfter = Duration.ofMillis(100);
+      final Duration increment = Duration.ofMillis(100);
+
+      Retry retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration)
+          .retryAfter(retryAfter).incrementBy(increment).maxWait(Duration.ofMillis(1000))
+          .backOffFactor(1.0).logInterval(Duration.ofMinutes(3)).createRetry();
+
+      // the max retries should be calculated like this:
+      // 1. 100
+      // 2. 100 + 100 = 200
+      // 3. 200 + 100 = 300
+      // 4. 300 + 100 = 400
+      // 5. 400 + 100 = 500
+
+      // 100 + 200 + 300 + 400 + 500 = 1500
+
+      assertEquals(5, retry.getMaxRetries());
+    }
+
+    @Test
+    public void withBackoffFactorAndMaxWait() {
+      final Duration retriesForDuration = Duration.ofSeconds(4);
+      final Duration retryAfter = Duration.ofMillis(100);
+      Retry retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration)
+          .retryAfter(retryAfter).incrementBy(Duration.ofMillis(0)).maxWait(Duration.ofMillis(500))
+          .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry();
+
+      // max retries should be calculated like this:
+      // 1. 100
+      // 2. 100 * 1.5 = 150
+      // 3. 150 * 1.5 = 225
+      // 4. 225 * 1.5 = 337
+      // 5. 337 * 1.5 = 505 (which is greater than the max wait of 500 so its capped)
+
+      // 100 + 150 + 225 + 337 + 500 + 500 + 500 + 500 + 500 + 500 = 3812
+      assertEquals(10, retry.getMaxRetries());
+    }
+
+    @Test
+    public void smallDuration() {
+      Duration retriesForDuration = Duration.ofMillis(0);
+      final Duration retryAfter = Duration.ofMillis(100);
+      Retry retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration)
+          .retryAfter(retryAfter).incrementBy(Duration.ofMillis(0)).maxWait(Duration.ofMillis(500))
+          .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry();
+      assertEquals(0, retry.getMaxRetries());
+
+      retriesForDuration = Duration.ofMillis(99);
+      assertTrue(retriesForDuration.compareTo(retryAfter) < 0);
+      retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration).retryAfter(retryAfter)
+          .incrementBy(Duration.ofMillis(0)).maxWait(Duration.ofMillis(500)).backOffFactor(1.5)
+          .logInterval(Duration.ofMinutes(3)).createRetry();
+      assertEquals(0, retry.getMaxRetries());
+    }
+
+    @Test
+    public void equalDurationAndInitialWait() {
+      final Duration retriesForDuration = Duration.ofMillis(100);
+      final Duration retryAfter = Duration.ofMillis(100);
+      assertEquals(0, retriesForDuration.compareTo(retryAfter));
+      Retry retry = Retry.builder().maxRetriesWithinDuration(retriesForDuration)
+          .retryAfter(retryAfter).incrementBy(Duration.ofMillis(0)).maxWait(Duration.ofMillis(500))
+          .backOffFactor(1.5).logInterval(Duration.ofMinutes(3)).createRetry();
+      assertEquals(1, retry.getMaxRetries());
+    }
+  }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java b/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java
index f1d3c03..9bc5c7d 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/ValidatorsTest.java
@@ -28,20 +28,12 @@
 
 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.commons.lang3.StringUtils;
 import org.junit.jupiter.api.Test;
 
 public class ValidatorsTest {
 
-  @SuppressWarnings("deprecation")
-  private static final TableId REPL_TABLE_ID =
-      org.apache.accumulo.core.replication.ReplicationTable.ID;
-  @SuppressWarnings("deprecation")
-  private static final String REPL_TABLE_NAME =
-      org.apache.accumulo.core.replication.ReplicationTable.NAME;
-
   private static <T> void checkNull(Consumer<T> nullConsumer) {
     var e = assertThrows(IllegalArgumentException.class, () -> nullConsumer.accept(null));
     assertTrue(e.getMessage().endsWith("must not be null"));
@@ -62,8 +54,8 @@
   public void test_CAN_CLONE_TABLE() {
     Validator<TableId> v = Validators.CAN_CLONE_TABLE;
     checkNull(v::validate);
-    assertAllValidate(v, List.of(REPL_TABLE_ID, TableId.of("id1")));
-    assertAllThrow(v, List.of(RootTable.ID, MetadataTable.ID));
+    assertAllValidate(v, List.of(TableId.of("id1")));
+    assertAllThrow(v, List.of(AccumuloTable.ROOT.tableId(), AccumuloTable.METADATA.tableId()));
   }
 
   @Test
@@ -80,8 +72,8 @@
     Validator<String> v = Validators.EXISTING_TABLE_NAME;
     checkNull(v::validate);
     assertAllValidate(v,
-        List.of(RootTable.NAME, MetadataTable.NAME, "normalTable", "withNumber2", "has_underscore",
-            "_underscoreStart", StringUtils.repeat("a", 1025),
+        List.of(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName(), "normalTable",
+            "withNumber2", "has_underscore", "_underscoreStart", StringUtils.repeat("a", 1025),
             StringUtils.repeat("a", 1025) + "." + StringUtils.repeat("a", 1025)));
     assertAllThrow(v, List.of("has-dash", "has-dash.inNamespace", "has.dash-inTable", " hasSpace",
         ".", "has$dollar", "two.dots.here", ".startsDot"));
@@ -102,8 +94,8 @@
     Validator<String> v = Validators.NEW_TABLE_NAME;
     checkNull(v::validate);
     assertAllValidate(v,
-        List.of(RootTable.NAME, MetadataTable.NAME, "normalTable", "withNumber2", "has_underscore",
-            "_underscoreStart", StringUtils.repeat("a", 1024),
+        List.of(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName(), "normalTable",
+            "withNumber2", "has_underscore", "_underscoreStart", StringUtils.repeat("a", 1024),
             StringUtils.repeat("a", 1025) + "." + StringUtils.repeat("a", 1024)));
     assertAllThrow(v,
         List.of("has-dash", "has-dash.inNamespace", "has.dash-inTable", " hasSpace", ".",
@@ -124,15 +116,15 @@
     Validator<String> v = Validators.NOT_BUILTIN_TABLE;
     checkNull(v::validate);
     assertAllValidate(v, List.of("root", "metadata", "user", "ns1.table2"));
-    assertAllThrow(v, List.of(RootTable.NAME, MetadataTable.NAME, REPL_TABLE_NAME));
+    assertAllThrow(v, List.of(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName()));
   }
 
   @Test
   public void test_NOT_METADATA_TABLE() {
     Validator<String> v = Validators.NOT_METADATA_TABLE;
     checkNull(v::validate);
-    assertAllValidate(v, List.of("root", "metadata", "user", "ns1.table2", REPL_TABLE_NAME));
-    assertAllThrow(v, List.of(RootTable.NAME, MetadataTable.NAME));
+    assertAllValidate(v, List.of("root", "metadata", "user", "ns1.table2"));
+    assertAllThrow(v, List.of(AccumuloTable.ROOT.tableName(), AccumuloTable.METADATA.tableName()));
   }
 
   @Test
@@ -140,16 +132,16 @@
     Validator<TableId> v = Validators.NOT_ROOT_TABLE_ID;
     checkNull(v::validate);
     assertAllValidate(v,
-        List.of(TableId.of(""), MetadataTable.ID, REPL_TABLE_ID, TableId.of(" #0(U!$. ")));
-    assertAllThrow(v, List.of(RootTable.ID));
+        List.of(TableId.of(""), AccumuloTable.METADATA.tableId(), TableId.of(" #0(U!$. ")));
+    assertAllThrow(v, List.of(AccumuloTable.ROOT.tableId()));
   }
 
   @Test
   public void test_VALID_TABLE_ID() {
     Validator<TableId> v = Validators.VALID_TABLE_ID;
     checkNull(v::validate);
-    assertAllValidate(v, List.of(RootTable.ID, MetadataTable.ID, REPL_TABLE_ID, TableId.of("111"),
-        TableId.of("aaaa"), TableId.of("r2d2")));
+    assertAllValidate(v, List.of(AccumuloTable.ROOT.tableId(), AccumuloTable.METADATA.tableId(),
+        TableId.of("111"), TableId.of("aaaa"), TableId.of("r2d2")));
     assertAllThrow(v, List.of(TableId.of(""), TableId.of("#0(U!$"), TableId.of(" #0(U!$. "),
         TableId.of("."), TableId.of(" "), TableId.of("C3P0")));
   }
diff --git a/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionPrioritizerTest.java b/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionPrioritizerTest.java
index 70f18be..5eaad1d 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionPrioritizerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionPrioritizerTest.java
@@ -74,12 +74,10 @@
     var j3 = createJob(CompactionKind.USER, "t-011", 11, 20);
     var j4 = createJob(CompactionKind.SYSTEM, "t-012", 11, 30);
     var j5 = createJob(CompactionKind.SYSTEM, "t-013", 5, 10);
-    var j6 = createJob(CompactionKind.CHOP, "t-014", 5, 40);
-    var j7 = createJob(CompactionKind.CHOP, "t-015", 5, 7);
-    var j8 = createJob(CompactionKind.SELECTOR, "t-014", 5, 21);
-    var j9 = createJob(CompactionKind.SELECTOR, "t-015", 7, 20);
+    var j8 = createJob(DeprecatedCompactionKind.SELECTOR, "t-014", 5, 21);
+    var j9 = createJob(DeprecatedCompactionKind.SELECTOR, "t-015", 7, 20);
 
-    var expected = List.of(j6, j2, j3, j1, j7, j4, j9, j8, j5);
+    var expected = List.of(j2, j3, j1, j4, j9, j8, j5);
 
     var shuffled = new ArrayList<>(expected);
     Collections.shuffle(shuffled);
diff --git a/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfigTest.java b/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfigTest.java
new file mode 100644
index 0000000..57eb2e3
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfigTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.compaction;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+import org.junit.jupiter.api.Test;
+
+public class CompactionServicesConfigTest {
+
+  @SuppressWarnings("deprecation")
+  private final Property oldPrefix = Property.TSERV_COMPACTION_SERVICE_PREFIX;
+  private final Property newPrefix = Property.COMPACTION_SERVICE_PREFIX;
+
+  @Test
+  public void testCompactionProps() {
+    ConfigurationCopy conf = new ConfigurationCopy();
+
+    conf.set(newPrefix.getKey() + "default.planner", DefaultCompactionPlanner.class.getName());
+    conf.set(newPrefix.getKey() + "default.planner.opts.maxOpen", "10");
+    conf.set(newPrefix.getKey() + "default.planner.opts.executors",
+        "[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'medium','type':'internal','maxSize':'128M','numThreads':2},{'name':'large','type':'internal','numThreads':2}]");
+
+    conf.set(oldPrefix.getKey() + "default.planner.opts.ignoredProp", "1");
+    conf.set(newPrefix.getKey() + "default.planner.opts.validProp", "1");
+    conf.set(oldPrefix.getKey() + "default.planner.opts.validProp", "a");
+
+    var compactionConfig = new CompactionServicesConfig(conf);
+    assertEquals(Map.of("maxOpen", "10", "executors",
+        "[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'medium','type':'internal','maxSize':'128M','numThreads':2},{'name':'large','type':'internal','numThreads':2}]",
+        "validProp", "1"), compactionConfig.getOptions().get("default"));
+  }
+
+  @Test
+  public void testDuplicateCompactionPlannerDefs() {
+    ConfigurationCopy conf = new ConfigurationCopy();
+
+    String planner = DefaultCompactionPlanner.class.getName();
+    String oldPlanner = "OldPlanner";
+
+    conf.set(newPrefix.getKey() + "default.planner", planner);
+    conf.set(oldPrefix.getKey() + "default.planner", oldPlanner);
+
+    conf.set(oldPrefix.getKey() + "old.planner", oldPlanner);
+
+    var compactionConfig = new CompactionServicesConfig(conf);
+    assertEquals(Map.of("default", planner, "old", oldPlanner), compactionConfig.getPlanners());
+  }
+
+  @Test
+  public void testCompactionPlannerOldDef() {
+    ConfigurationCopy conf = new ConfigurationCopy();
+
+    conf.set(oldPrefix.getKey() + "cs1.planner", DefaultCompactionPlanner.class.getName());
+    conf.set(oldPrefix.getKey() + "cs1.planner.opts.maxOpen", "10");
+    conf.set(oldPrefix.getKey() + "cs1.planner.opts.executors",
+        "[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'medium','type':'internal','maxSize':'128M','numThreads':2},{'name':'large','type':'internal','numThreads':2}]");
+    conf.set(oldPrefix.getKey() + "cs1.planner.opts.foo", "1");
+
+    var compactionConfig = new CompactionServicesConfig(conf);
+    assertTrue(compactionConfig.getOptions().get("cs1").containsKey("foo"));
+    assertEquals("1", compactionConfig.getOptions().get("cs1").get("foo"));
+  }
+
+  @Test
+  public void testCompactionRateLimits() {
+    ConfigurationCopy conf = new ConfigurationCopy();
+    CompactionServicesConfig compactionConfig;
+
+    conf.set(oldPrefix.getKey() + "cs1.planner", DefaultCompactionPlanner.class.getName());
+    conf.set(oldPrefix.getKey() + "cs1.rate.limit", "2M");
+    compactionConfig = new CompactionServicesConfig(conf);
+    assertEquals(2097152, compactionConfig.getRateLimits().get("cs1"));
+
+    // Test service collision
+    conf.set(newPrefix.getKey() + "cs1.rate.limit", "4M");
+    var e = assertThrows(IllegalArgumentException.class, () -> new CompactionServicesConfig(conf),
+        "failed to throw error");
+    assertEquals("Incomplete compaction service definition, missing planner class: cs1.rate.limit",
+        e.getMessage(), "Error message was not equal");
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.java b/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.java
deleted file mode 100644
index 5702568..0000000
--- a/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.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.util.format;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Map;
-import java.util.TimeZone;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("deprecation")
-public class DateStringFormatterTest {
-  DateStringFormatter formatter;
-
-  Map<Key,Value> data;
-
-  @BeforeEach
-  public void setUp() {
-    formatter = new DateStringFormatter();
-    data = new TreeMap<>();
-    data.put(new Key("", "", "", 0), new Value());
-  }
-
-  private void testFormatterIgnoresConfig(FormatterConfig config, DateStringFormatter formatter) {
-    // ignores config's DateFormatSupplier and substitutes its own
-    formatter.initialize(data.entrySet(), config);
-
-    assertTrue(formatter.hasNext());
-    final String next = formatter.next();
-    assertTrue(next.endsWith("1970/01/01 00:00:00.000"), next);
-  }
-
-  @Test
-  public void testTimestamps() {
-    final TimeZone utc = TimeZone.getTimeZone("UTC");
-    final TimeZone est = TimeZone.getTimeZone("EST");
-    final FormatterConfig config = new FormatterConfig().setPrintTimestamps(true);
-    DateStringFormatter formatter;
-
-    formatter = new DateStringFormatter(utc);
-    testFormatterIgnoresConfig(config, formatter);
-
-    // even though config says to use EST and only print year, the Formatter will override these
-    formatter = new DateStringFormatter(utc);
-    DateFormatSupplier dfSupplier = DateFormatSupplier.createSimpleFormatSupplier("YYYY", est);
-    config.setDateFormatSupplier(dfSupplier);
-    testFormatterIgnoresConfig(config, formatter);
-  }
-
-  @Test
-  public void testNoTimestamps() {
-    data.put(new Key("", "", "", 1), new Value());
-
-    assertEquals(2, data.size());
-
-    formatter.initialize(data.entrySet(), new FormatterConfig());
-
-    assertEquals(formatter.next(), formatter.next());
-  }
-
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/util/format/HexFormatterTest.java b/core/src/test/java/org/apache/accumulo/core/util/format/HexFormatterTest.java
deleted file mode 100644
index d276d10..0000000
--- a/core/src/test/java/org/apache/accumulo/core/util/format/HexFormatterTest.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.format;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.1.0")
-public class HexFormatterTest {
-  HexFormatter formatter;
-
-  Map<Key,Value> data;
-
-  @BeforeEach
-  public void setUp() {
-    data = new TreeMap<>();
-    formatter = new HexFormatter();
-  }
-
-  @Test
-  public void testInitialize() {
-    data.put(new Key(), new Value());
-    formatter.initialize(data.entrySet(), new FormatterConfig());
-
-    assertTrue(formatter.hasNext());
-    assertEquals("     [] ", formatter.next());
-  }
-
-  @Test
-  public void testInterpretRow() {
-    assertEquals(new Text(), formatter.interpretRow(new Text()));
-    assertEquals(new Text("\0"), formatter.interpretRow(new Text("0")));
-  }
-
-  @Test
-  public void testRoundTripRows() {
-    Text bytes = new Text(new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15});
-    data.put(new Key(bytes), new Value());
-
-    formatter.initialize(data.entrySet(), new FormatterConfig());
-
-    String row = formatter.next().split(" ")[0];
-    assertEquals("0001-0203-0405-0607-0809-0a0b-0c0d-0e0f", row);
-    assertEquals(bytes, formatter.interpretRow(new Text(row)));
-  }
-
-  @Test
-  public void testInterpretBadRow0() {
-    assertThrows(IllegalArgumentException.class, () -> formatter.interpretRow(new Text("!")));
-  }
-
-  @Test
-  public void testInterpretBadRow1() {
-    assertThrows(IllegalArgumentException.class, () -> formatter.interpretRow(new Text("z")));
-  }
-
-  @Test
-  public void testTimestamps() {
-    long now = System.currentTimeMillis();
-    data.put(new Key("", "", "", now), new Value());
-    formatter.initialize(data.entrySet(), new FormatterConfig().setPrintTimestamps(true));
-    String entry = formatter.next().split("\\s+")[2];
-    assertEquals(now, Long.parseLong(entry));
-  }
-}
diff --git a/core/src/test/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapterTest.java b/core/src/test/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapterTest.java
new file mode 100644
index 0000000..d9d84bd
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/json/ByteArrayToBase64TypeAdapterTest.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.util.json;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+
+import com.google.gson.Gson;
+
+public class ByteArrayToBase64TypeAdapterTest {
+
+  private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
+
+  @Test
+  public void testSerializeText() throws IOException {
+    final Text original = new Text("This is a test");
+
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos)) {
+      original.write(new DataOutputStream(dos));
+
+      final String encoded = gson.toJson(baos.toByteArray());
+      final Text decoded = new Text();
+      decoded.readFields(
+          new DataInputStream(new ByteArrayInputStream(gson.fromJson(encoded, byte[].class))));
+      assertEquals(original.toString(), decoded.toString());
+    }
+  }
+
+  @Test
+  public void testByteArray() {
+    final byte[] original = new byte[] {0x01, 0x06, 0x34, 0x09, 0x12, 0x34, 0x57, 0x56, 0x30, 0x74};
+
+    final String encoded = gson.toJson(original);
+    final byte[] decoded = gson.fromJson(encoded, byte[].class);
+
+    assertArrayEquals(original, decoded);
+  }
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/util/time/NanoTimeTest.java b/core/src/test/java/org/apache/accumulo/core/util/time/NanoTimeTest.java
new file mode 100644
index 0000000..d306aaf
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/time/NanoTimeTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+
+public class NanoTimeTest {
+  @Test
+  public void testMultipleTimes() {
+    List<NanoTime> ntimes = new ArrayList<>();
+
+    NanoTime prev = NanoTime.now();
+    ntimes.add(prev);
+
+    for (int i = 0; i < 100; i++) {
+      NanoTime next = NanoTime.now();
+      while (prev.equals(next)) {
+        next = NanoTime.now();
+      }
+
+      ntimes.add(next);
+      prev = next;
+    }
+
+    long curr = System.nanoTime();
+    while (curr == System.nanoTime()) {}
+
+    var start = NanoTime.now();
+
+    while (start.equals(NanoTime.now())) {}
+
+    for (int i = 1; i < ntimes.size(); i++) {
+      var last = ntimes.get(i - 1);
+      var next = ntimes.get(i);
+      assertTrue(last.compareTo(next) < 0);
+      assertTrue(next.compareTo(last) > 0);
+      assertTrue(next.compareTo(next) == 0);
+      assertTrue(next.elapsed().toNanos() > 0);
+      assertEquals(next, next);
+      assertEquals(next.hashCode(), next.hashCode());
+      assertNotEquals(last, next);
+      assertNotEquals(last.hashCode(), next.hashCode());
+
+      var duration1 = next.elapsed();
+      var duration2 = start.subtract(last);
+      var duration3 = start.subtract(next);
+
+      assertTrue(duration2.compareTo(duration3) > 0);
+      assertTrue(duration1.compareTo(duration3) > 0);
+    }
+
+    var copy = List.copyOf(ntimes);
+    Collections.shuffle(ntimes);
+    Collections.sort(ntimes);
+    assertEquals(copy, ntimes);
+  }
+
+  @Test
+  public void testBoundry() {
+    // tests crossing the Long.MAX_VALUE boundry
+    long origin = Long.MAX_VALUE - 1000;
+
+    List<NanoTime> ntimes = new ArrayList<>();
+
+    // add times that start positive and then go negative
+    for (int i = 0; i < 20; i++) {
+      var nt = i * 100 + origin;
+      ntimes.add(new NanoTime(nt));
+    }
+
+    for (int i = 1; i < ntimes.size(); i++) {
+      var last = ntimes.get(i - 1);
+      var next = ntimes.get(i);
+      assertEquals(100, next.subtract(last).toNanos());
+      assertEquals(-100, last.subtract(next).toNanos());
+      assertTrue(next.compareTo(last) > 0);
+      assertTrue(last.compareTo(next) < 0);
+      assertTrue(next.compareTo(next) == 0);
+    }
+
+    var copy = List.copyOf(ntimes);
+    Collections.shuffle(ntimes);
+    Collections.sort(ntimes);
+    assertEquals(copy, ntimes);
+  }
+
+  @Test
+  public void testNowPlus() {
+
+    List<NanoTime> ntimes = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      ntimes.add(NanoTime.nowPlus(Duration.ofHours(i)));
+    }
+
+    for (int i = 1; i < ntimes.size(); i++) {
+      var last = ntimes.get(i - 1);
+      var next = ntimes.get(i);
+
+      var duration = next.subtract(last);
+
+      assertTrue(duration.compareTo(Duration.ofHours(1)) >= 0);
+      // This could fail if the test process were paused for more than 3 minutes
+      assertTrue(duration.compareTo(Duration.ofMinutes(63)) < 0);
+      assertTrue(next.elapsed().compareTo(Duration.ZERO) < 0);
+    }
+
+    var copy = List.copyOf(ntimes);
+    Collections.shuffle(ntimes);
+    Collections.sort(ntimes);
+    assertEquals(copy, ntimes);
+
+    ntimes.clear();
+
+    // nano time can compute elapsed times in a 290 year period which should wrap Long.MAX_VALUE no
+    // matter where it starts
+    for (int i = 0; i < 290; i++) {
+      ntimes.add(NanoTime.nowPlus(Duration.ofDays(365 * i)));
+    }
+
+    for (int i = 1; i < ntimes.size(); i++) {
+      var last = ntimes.get(i - 1);
+      var next = ntimes.get(i);
+
+      var duration = next.subtract(last);
+
+      assertTrue(duration.compareTo(Duration.ofDays(365)) >= 0);
+      assertTrue(duration.compareTo(Duration.ofDays(366)) < 0);
+      assertTrue(next.elapsed().compareTo(Duration.ZERO) < 0);
+    }
+
+    copy = List.copyOf(ntimes);
+    Collections.shuffle(ntimes);
+    Collections.sort(ntimes);
+    assertEquals(copy, ntimes);
+  }
+
+}
diff --git a/core/src/test/java/org/apache/accumulo/core/util/time/SteadyTimeTest.java b/core/src/test/java/org/apache/accumulo/core/util/time/SteadyTimeTest.java
new file mode 100644
index 0000000..016c771
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/time/SteadyTimeTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.jupiter.api.Test;
+
+public class SteadyTimeTest {
+
+  @Test
+  public void testSteadyTime() {
+    long time = 20_000;
+    var steadyTime = SteadyTime.from(time, TimeUnit.NANOSECONDS);
+
+    assertEquals(time, steadyTime.getNanos());
+    assertEquals(TimeUnit.NANOSECONDS.toMillis(time), steadyTime.getMillis());
+    assertEquals(Duration.ofNanos(time), steadyTime.getDuration());
+
+    // Verify equals and compareTo work correctly for same
+    var steadyTime2 = SteadyTime.from(time, TimeUnit.NANOSECONDS);
+    assertEquals(steadyTime, steadyTime2);
+    assertEquals(0, steadyTime.compareTo(steadyTime2));
+
+    // Check equals/compareto different objects
+    var steadyTime3 = SteadyTime.from(time + 100, TimeUnit.NANOSECONDS);
+    assertNotEquals(steadyTime, steadyTime3);
+    assertTrue(steadyTime.compareTo(steadyTime3) < 1);
+
+    // Negatives are not allowed
+    assertThrows(IllegalArgumentException.class, () -> SteadyTime.from(-100, TimeUnit.NANOSECONDS));
+  }
+
+}
diff --git a/core/src/test/resources/log4j2-test.properties b/core/src/test/resources/log4j2-test.properties
index b5ec989..1bd851a 100644
--- a/core/src/test/resources/log4j2-test.properties
+++ b/core/src/test/resources/log4j2-test.properties
@@ -36,9 +36,6 @@
 logger.03.name = org.apache.accumulo.core.iterators.user.TransformingIteratorTest$IllegalVisKeyTransformingIterator
 logger.03.level = fatal
 
-logger.04.name = org.apache.commons.vfs2.impl.DefaultFileSystemManager
-logger.04.level = warn
-
 logger.05.name = org.apache.hadoop.mapred
 logger.05.level = error
 
@@ -59,4 +56,3 @@
 
 rootLogger.level = info
 rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/hadoop-mapreduce/pom.xml b/hadoop-mapreduce/pom.xml
index a56abf9..2d67c15 100644
--- a/hadoop-mapreduce/pom.xml
+++ b/hadoop-mapreduce/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-hadoop-mapreduce</artifactId>
   <name>Apache Accumulo Hadoop MapReduce</name>
@@ -95,7 +95,7 @@
             </goals>
             <configuration>
               <includes>
-                <include>org[.]apache[.]accumulo[.]core[.]client[.]mapred(?:uce)?[.].*</include>
+                <include>org[.]apache[.]accumulo[.]hadoop[.]mapred(?:uce)?[.].*</include>
               </includes>
               <excludes>
                 <exclude>.*[.]impl[.].*</exclude>
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/partition/RangePartitioner.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/partition/RangePartitioner.java
index ac1ca8b..73a82a9 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/partition/RangePartitioner.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoop/mapreduce/partition/RangePartitioner.java
@@ -27,7 +27,7 @@
 import java.util.Scanner;
 import java.util.TreeSet;
 
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.DistributedCacheHelper;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
index b9440ee..09bb453 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AccumuloRecordReader.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.hadoopImpl.mapred;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -72,7 +72,6 @@
  */
 public abstract class AccumuloRecordReader<K,V> implements RecordReader<K,V> {
 
-  private static final SecureRandom random = new SecureRandom();
   // class to serialize configuration under in the job
   private final Class<?> CLASS;
   private static final Logger log = LoggerFactory.getLogger(AccumuloRecordReader.class);
@@ -337,7 +336,7 @@
             while (binnedRanges == null) {
               // Some tablets were still online, try again
               // sleep randomly between 100 and 200 ms
-              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
+              sleepUninterruptibly(100 + RANDOM.get().nextInt(100), TimeUnit.MILLISECONDS);
               binnedRanges = binOfflineTable(job, tableId, ranges, callingClass);
             }
           } else {
@@ -353,7 +352,7 @@
               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);
+              sleepUninterruptibly(100 + RANDOM.get().nextInt(100), TimeUnit.MILLISECONDS);
               tl.invalidateCache();
             }
           }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
index dc614d6..3b60ca7 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AccumuloRecordReader.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.hadoopImpl.mapreduce;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -72,7 +72,6 @@
  * the user's K/V types.
  */
 public abstract class AccumuloRecordReader<K,V> extends RecordReader<K,V> {
-  private static final SecureRandom random = new SecureRandom();
   private static final Logger log = LoggerFactory.getLogger(AccumuloRecordReader.class);
   // class to serialize configuration under in the job
   private final Class<?> CLASS;
@@ -369,7 +368,7 @@
             while (binnedRanges == null) {
               // Some tablets were still online, try again
               // sleep randomly between 100 and 200 ms
-              sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
+              sleepUninterruptibly(100 + RANDOM.get().nextInt(100), TimeUnit.MILLISECONDS);
               binnedRanges = binOfflineTable(context, tableId, ranges, callingClass);
 
             }
@@ -386,7 +385,7 @@
               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);
+              sleepUninterruptibly(100 + RANDOM.get().nextInt(100), TimeUnit.MILLISECONDS);
               tl.invalidateCache();
             }
           }
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
index 16d1324..22a280c 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/ConfiguratorBase.java
@@ -30,7 +30,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/DistributedCacheHelper.java
similarity index 97%
rename from core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java
rename to hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/DistributedCacheHelper.java
index 90f6691..9569a36 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/DistributedCacheHelper.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
+package org.apache.accumulo.hadoopImpl.mapreduce.lib;
 
 import static java.util.Objects.requireNonNull;
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
index b013aa7..3f23b9f 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/lib/InputConfigurator.java
@@ -63,7 +63,7 @@
 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.AccumuloTable;
 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;
@@ -855,7 +855,8 @@
 
       Range metadataRange =
           new Range(new KeyExtent(tableId, startRow, null).toMetaRow(), true, null, false);
-      Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      Scanner scanner =
+          context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       scanner.fetchColumnFamily(LastLocationColumnFamily.NAME);
       scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
diff --git a/hadoop-mapreduce/src/main/spotbugs/exclude-filter.xml b/hadoop-mapreduce/src/main/spotbugs/exclude-filter.xml
index c5b24cd..358f5e7 100644
--- a/hadoop-mapreduce/src/main/spotbugs/exclude-filter.xml
+++ b/hadoop-mapreduce/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>
     <!-- More convenient to ignore these everywhere, because it's very common and unimportant -->
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
index bcebf00..5bff999 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
@@ -37,6 +37,7 @@
 import org.apache.accumulo.core.data.Value;
 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.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
@@ -197,8 +198,9 @@
 
       Configuration conf = cluster.getServerContext().getHadoopConf();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
+      FileSystem fileSystem = FileSystem.getLocal(conf);
       FileSKVIterator sample = FileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
+          .forFile(UnreferencedTabletFile.of(fileSystem, files[0]), fileSystem, conf,
               NoCryptoServiceFactory.NONE)
           .withTableConfiguration(acuconf).build()
           .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
index 287f415..43e1839 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloFileOutputFormatIT.java
@@ -37,6 +37,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.rfile.RFileOperations;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
@@ -212,9 +213,10 @@
 
       Configuration conf = cluster.getServerContext().getHadoopConf();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
+      FileSystem fs = FileSystem.getLocal(conf);
       FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
-              NoCryptoServiceFactory.NONE)
+          .forFile(UnreferencedTabletFile.of(fs, new Path(files[0].toString())),
+              FileSystem.getLocal(conf), conf, NoCryptoServiceFactory.NONE)
           .withTableConfiguration(acuconf).build()
           .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
       assertNotNull(sample);
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
index 559c649..b79db6c 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/AccumuloInputFormatIT.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.hadoop.its.mapreduce;
 
 import static java.lang.System.currentTimeMillis;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -33,7 +32,6 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -125,7 +123,7 @@
       splitsToAdd.add(new Text(String.format("%09d", i)));
     }
     client.tableOperations().addSplits(table, splitsToAdd);
-    sleepUninterruptibly(500, TimeUnit.MILLISECONDS); // wait for splits to be propagated
+    Thread.sleep(500); // wait for splits to be propagated
 
     // get splits without setting any range
     // No ranges set on the job so it'll start with -inf
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/MapReduceIT.java
similarity index 88%
rename from test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
rename to hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/MapReduceIT.java
index 1589f62..52b66fa 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapreduce/MapReduceIT.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.test.mapreduce;
+package org.apache.accumulo.hadoop.its.mapreduce;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
@@ -48,10 +48,6 @@
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
 public class MapReduceIT extends ConfigurableMacBase {
 
   public static final String hadoopTmpDirArg =
@@ -62,7 +58,6 @@
   static final String input_cq = "cq-NOTHASHED";
   static final String input_cfcq = input_cf + ":" + input_cq;
   static final String output_cq = "cq-MD4BASE64";
-  static final String output_cfcq = input_cf + ":" + output_cq;
 
   @Override
   protected Duration defaultTimeout() {
@@ -71,7 +66,8 @@
 
   @Test
   public void test() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
+    var props = getClientProperties();
+    try (AccumuloClient client = Accumulo.newClient().from(props).build()) {
       runTest(client, getCluster());
     }
   }
@@ -90,8 +86,8 @@
     }
     bw.close();
 
-    Process hash = cluster.exec(RowHash.class, Collections.singletonList(hadoopTmpDirArg), "-c",
-        cluster.getClientPropsPath(), "-t", tablename, "--column", input_cfcq).getProcess();
+    Process hash = cluster.exec(RowHashIT.RowHash.class, Collections.singletonList(hadoopTmpDirArg),
+        "-c", cluster.getClientPropsPath(), "-t", tablename, "--column", input_cfcq).getProcess();
     assertEquals(0, hash.waitFor());
 
     try (Scanner s = c.createScanner(tablename, Authorizations.EMPTY)) {
diff --git a/hadoop-mapreduce/src/test/resources/log4j2-test.properties b/hadoop-mapreduce/src/test/resources/log4j2-test.properties
index ea2309d..2bd9e20 100644
--- a/hadoop-mapreduce/src/test/resources/log4j2-test.properties
+++ b/hadoop-mapreduce/src/test/resources/log4j2-test.properties
@@ -36,9 +36,6 @@
 logger.03.name = org.apache.accumulo.core.iterators.user.TransformingIteratorTest$IllegalVisKeyTransformingIterator
 logger.03.level = fatal
 
-logger.04.name = org.apache.commons.vfs2.impl.DefaultFileSystemManager
-logger.04.level = warn
-
 logger.05.name = org.apache.hadoop.mapred
 logger.05.level = error
 
@@ -59,4 +56,3 @@
 
 rootLogger.level = info
 rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/iterator-test-harness/pom.xml b/iterator-test-harness/pom.xml
index 11fbcb2..e142e28 100644
--- a/iterator-test-harness/pom.xml
+++ b/iterator-test-harness/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-iterator-test-harness</artifactId>
   <name>Apache Accumulo Iterator Test Harness</name>
diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java
index 3cadb36..8798b52 100644
--- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java
+++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/IsolatedDeepCopiesTestCase.java
@@ -18,8 +18,9 @@
  */
 package org.apache.accumulo.iteratortest.testcases;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -43,8 +44,6 @@
 public class IsolatedDeepCopiesTestCase implements IteratorTestCase {
   private static final Logger log = LoggerFactory.getLogger(IsolatedDeepCopiesTestCase.class);
 
-  private static final SecureRandom random = new SecureRandom();
-
   @Override
   public IteratorTestOutput test(IteratorTestInput testInput) {
     final SortedKeyValueIterator<Key,Value> skvi = IteratorTestUtil.instantiateIterator(testInput);
@@ -81,7 +80,7 @@
     // All of the copies should have consistent results from concurrent use
     while (allHasTop(iterators)) {
       // occasionally deep copy one of the existing iterators
-      if (random.nextInt(3) == 0) {
+      if (RANDOM.get().nextInt(3) == 0) {
         log.debug("Deep-copying and re-seeking an iterator");
         SortedKeyValueIterator<Key,Value> newcopy = getRandomElement(iterators).deepCopy(iterEnv);
         newcopy.seek(
@@ -109,7 +108,7 @@
     if (iterators == null || iterators.isEmpty()) {
       throw new IllegalArgumentException("should not pass an empty collection");
     }
-    int num = random.nextInt(iterators.size());
+    int num = RANDOM.get().nextInt(iterators.size());
     for (E e : iterators) {
       if (num-- == 0) {
         return e;
diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java
index e1010d7..cde5a93 100644
--- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java
+++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/MultipleHasTopCalls.java
@@ -18,8 +18,9 @@
  */
 package org.apache.accumulo.iteratortest.testcases;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.data.Key;
@@ -39,8 +40,6 @@
  */
 public class MultipleHasTopCalls implements IteratorTestCase {
 
-  private static final SecureRandom random = new SecureRandom();
-
   @Override
   public IteratorTestOutput test(IteratorTestInput testInput) {
     final SortedKeyValueIterator<Key,Value> skvi = IteratorTestUtil.instantiateIterator(testInput);
@@ -59,7 +58,7 @@
     TreeMap<Key,Value> data = new TreeMap<>();
     while (skvi.hasTop()) {
       // Check 1 to 5 times. If hasTop returned true, it should continue to return true.
-      for (int i = 0; i < random.nextInt(5) + 1; i++) {
+      for (int i = 0; i < RANDOM.get().nextInt(5) + 1; i++) {
         if (!skvi.hasTop()) {
           throw badStateException(true);
         }
@@ -70,7 +69,7 @@
     }
 
     // Check 1 to 5 times. Once hasTop returned false, it should continue to return false
-    for (int i = 0; i < random.nextInt(5) + 1; i++) {
+    for (int i = 0; i < RANDOM.get().nextInt(5) + 1; i++) {
       if (skvi.hasTop()) {
         throw badStateException(false);
       }
diff --git a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java
index a537c25..5a4b868 100644
--- a/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java
+++ b/iterator-test-harness/src/main/java/org/apache/accumulo/iteratortest/testcases/ReSeekTestCase.java
@@ -18,8 +18,9 @@
  */
 package org.apache.accumulo.iteratortest.testcases;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.Collection;
 import java.util.TreeMap;
 
@@ -38,7 +39,6 @@
  * Test case that verifies that an iterator can use the generated instance from {@code deepCopy}.
  */
 public class ReSeekTestCase implements IteratorTestCase {
-  private static final SecureRandom random = new SecureRandom();
   private static final Logger log = LoggerFactory.getLogger(ReSeekTestCase.class);
 
   /**
@@ -67,7 +67,7 @@
     final Range origRange = testInput.getRange();
     final Collection<ByteSequence> origFamilies = testInput.getFamilies();
     final boolean origInclusive = testInput.isInclusive();
-    int reseekCount = random.nextInt(RESEEK_INTERVAL);
+    int reseekCount = RANDOM.get().nextInt(RESEEK_INTERVAL);
 
     int i = 0;
     while (skvi.hasTop()) {
diff --git a/minicluster/pom.xml b/minicluster/pom.xml
index 6a17606..ba9c8aa 100644
--- a/minicluster/pom.xml
+++ b/minicluster/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-minicluster</artifactId>
   <name>Apache Accumulo MiniCluster</name>
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
index f9f7efb..39e79a5 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -58,14 +58,6 @@
   AccumuloClient createAccumuloClient(String user, AuthenticationToken token);
 
   /**
-   * Get the client configuration for the cluster
-   *
-   * @deprecated since 2.0.0, replaced by {@link #getClientProperties()}}
-   */
-  @Deprecated(since = "2.0.0")
-  org.apache.accumulo.core.client.ClientConfiguration getClientConfig();
-
-  /**
    * @return client connection info for cluster
    */
   Properties getClientProperties();
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index b0441d4..155b97f 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -33,7 +33,6 @@
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientConfConverter;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -144,12 +143,6 @@
   }
 
   @Override
-  @Deprecated(since = "2.0.0")
-  public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
-    return ClientConfConverter.toClientConf(info.getProperties());
-  }
-
-  @Override
   public Properties getClientProperties() {
     return info.getProperties();
   }
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
index d146365..e594bed 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
@@ -160,7 +160,6 @@
   }
 
   @Override
-  @SuppressWarnings("removal")
   public void startAllServers(ServerType server) throws IOException {
     switch (server) {
       case TABLET_SERVER:
@@ -168,7 +167,6 @@
           start(server, tserver);
         }
         break;
-      case MASTER:
       case MANAGER:
         for (String manager : getHosts(MANAGER_HOSTS_FILE)) {
           start(server, manager);
@@ -209,7 +207,6 @@
   }
 
   @Override
-  @SuppressWarnings("removal")
   public void stopAllServers(ServerType server) throws IOException {
     switch (server) {
       case TABLET_SERVER:
@@ -217,7 +214,6 @@
           stop(server, tserver);
         }
         break;
-      case MASTER:
       case MANAGER:
         for (String manager : getHosts(MANAGER_HOSTS_FILE)) {
           stop(server, manager);
@@ -311,14 +307,12 @@
         "'{print \\$2}'", "|", "head", "-1", "|", "tr", "-d", "'\\n'"};
   }
 
-  @SuppressWarnings("removal")
   protected String getProcessString(ServerType server) {
     switch (server) {
       case TABLET_SERVER:
         return "tserver";
       case GARBAGE_COLLECTOR:
         return "gc";
-      case MASTER:
       case MANAGER:
         return "manager";
       case MONITOR:
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 2a583b6..b94449f 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -24,10 +24,7 @@
 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.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
@@ -127,19 +124,6 @@
   }
 
   /**
-   * Utility method to get a connector to the MAC.
-   *
-   * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #createAccumuloClient(String, AuthenticationToken)}
-   */
-  @Deprecated(since = "2.0.0")
-  public org.apache.accumulo.core.client.Connector getConnector(String user, String passwd)
-      throws AccumuloException, AccumuloSecurityException {
-    return org.apache.accumulo.core.client.Connector
-        .from(impl.createAccumuloClient(user, new PasswordToken(passwd)));
-  }
-
-  /**
    * Utility method to create an {@link AccumuloClient} with connection to the MAC. The
    * AccumuloClient object should be closed by user
    *
@@ -150,15 +134,6 @@
   }
 
   /**
-   * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #getClientProperties()}
-   */
-  @Deprecated(since = "2.0.0")
-  public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
-    return impl.getClientConfig();
-  }
-
-  /**
    * @return A copy of the connection properties for the cluster
    * @since 2.0.0
    */
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
index 2945eb7..4ee34fc 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloConfig.java
@@ -56,6 +56,7 @@
    *
    * @param numTservers the number of tablet servers that mini accumulo cluster should start
    */
+  @Deprecated(since = "3.1.0")
   public MiniAccumuloConfig setNumTservers(int numTservers) {
     impl.setNumTservers(numTservers);
     return this;
@@ -67,6 +68,7 @@
    * @param numScanServers the number of scan servers that mini accumulo cluster should start
    * @since 2.1.0
    */
+  @Deprecated(since = "3.1.0")
   public MiniAccumuloConfig setNumScanServers(int numScanServers) {
     impl.setNumScanServers(numScanServers);
     return this;
@@ -224,6 +226,7 @@
   /**
    * @return the number of tservers configured for this cluster
    */
+  @Deprecated(since = "3.1.0")
   public int getNumTservers() {
     return impl.getNumTservers();
   }
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
deleted file mode 100644
index c0dafae..0000000
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
+++ /dev/null
@@ -1,53 +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.minicluster;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-
-/**
- * @since 1.6.0
- * @deprecated since 2.0.0, Use {@link MiniAccumuloCluster#getClientProperties(File)} instead
- */
-@Deprecated(since = "2.0.0")
-public class MiniAccumuloInstance extends org.apache.accumulo.core.client.ZooKeeperInstance {
-
-  /**
-   * Construct an {@link org.apache.accumulo.core.client.Instance} entry point to Accumulo using a
-   * {@link MiniAccumuloCluster} directory
-   */
-  public MiniAccumuloInstance(String instanceName, File directory) throws FileNotFoundException {
-    super(org.apache.accumulo.core.client.ClientConfiguration
-        .fromFile(new File(new File(directory, "conf"), "client.conf")).withInstance(instanceName)
-        .withZkHosts(getZooKeepersFromDir(directory)));
-  }
-
-  // Keep this private to avoid bringing it into the public API
-  private static String getZooKeepersFromDir(File directory) {
-    if (!directory.isDirectory()) {
-      throw new IllegalArgumentException("Not a directory " + directory.getPath());
-    }
-    File configFile = new File(new File(directory, "conf"), "accumulo.properties");
-    var conf = SiteConfiguration.fromFile(configFile).build();
-    return conf.get(Property.INSTANCE_ZK_HOST);
-  }
-}
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
index b2333c3..cb4e9da 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
@@ -71,8 +71,6 @@
   private static final String ROOT_PASSWORD_PROP = "rootPassword";
   private static final String SHUTDOWN_PORT_PROP = "shutdownPort";
   private static final String DEFAULT_MEMORY_PROP = "defaultMemory";
-  @Deprecated(since = "2.1.0")
-  private static final String MASTER_MEMORY_PROP = "masterMemory";
   private static final String MANAGER_MEMORY_PROP = "managerMemory";
   private static final String TSERVER_MEMORY_PROP = "tserverMemory";
   private static final String ZOO_KEEPER_MEMORY_PROP = "zooKeeperMemory";
@@ -152,6 +150,7 @@
    *
    * @param args An optional -p argument can be specified with the path to a valid properties file.
    */
+  @SuppressWarnings("deprecation")
   @SuppressFBWarnings(value = {"PATH_TRAVERSAL_IN", "UNENCRYPTED_SERVER_SOCKET"},
       justification = "code runs in same security context as user who provided input file name; "
           + "socket need not be encrypted, since this class is provided for testing only")
@@ -206,10 +205,6 @@
       setMemoryOnConfig(config, opts.prop.getProperty(TSERVER_MEMORY_PROP),
           ServerType.TABLET_SERVER);
     }
-    if (opts.prop.containsKey(MASTER_MEMORY_PROP)) {
-      log.warn("{} is deprecated. Use {} instead.", MASTER_MEMORY_PROP, MANAGER_MEMORY_PROP);
-      setMemoryOnConfig(config, opts.prop.getProperty(MASTER_MEMORY_PROP), ServerType.MANAGER);
-    }
     if (opts.prop.containsKey(MANAGER_MEMORY_PROP)) {
       setMemoryOnConfig(config, opts.prop.getProperty(MANAGER_MEMORY_PROP), ServerType.MANAGER);
     }
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
index a695968..ab5c0d7 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
@@ -22,22 +22,12 @@
  * @since 1.6.0
  */
 public enum ServerType {
-  /**
-   * @deprecated Use {@link #MANAGER} instead
-   */
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  MASTER("Master"),
   MANAGER("Manager"),
   ZOOKEEPER("ZooKeeper"),
   TABLET_SERVER("TServer"),
   GARBAGE_COLLECTOR("GC"),
   COMPACTION_COORDINATOR("CompactionCoordinator"),
   COMPACTOR("Compactor"),
-  /**
-   * @deprecated Accumulo-managed Tracer service was removed
-   */
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  TRACER("Tracer"),
   MONITOR("Monitor"),
   SCAN_SERVER("SServer");
 
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
index 8844079..ff9e6d3 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
@@ -38,10 +38,10 @@
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
@@ -59,6 +59,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Maps;
+import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -173,7 +174,9 @@
       int count =
           Math.min(limit, cluster.getConfig().getNumCompactors() - compactorProcesses.size());
       for (int i = 0; i < count; i++) {
-        compactorProcesses.add(cluster.exec(compactor, "-q", queueName).getProcess());
+        compactorProcesses.add(
+            cluster.exec(compactor, "-o", Property.COMPACTOR_QUEUE_NAME.getKey() + "=" + queueName)
+                .getProcess());
       }
     }
   }
@@ -188,7 +191,6 @@
     start(server, Collections.emptyMap(), Integer.MAX_VALUE);
   }
 
-  @SuppressWarnings("removal")
   public synchronized void start(ServerType server, Map<String,String> configOverrides, int limit)
       throws IOException {
     if (limit <= 0) {
@@ -206,7 +208,6 @@
           }
         }
         break;
-      case MASTER:
       case MANAGER:
         if (managerProcess == null) {
           managerProcess = cluster._exec(Manager.class, server, configOverrides).getProcess();
@@ -261,10 +262,8 @@
   }
 
   @Override
-  @SuppressWarnings("removal")
   public synchronized void stop(ServerType server, String hostname) throws IOException {
     switch (server) {
-      case MASTER:
       case MANAGER:
         if (managerProcess != null) {
           try {
@@ -402,12 +401,10 @@
     throw new UnsupportedOperationException();
   }
 
-  @SuppressWarnings("removal")
   public void killProcess(ServerType type, ProcessReference procRef)
       throws ProcessNotFoundException, InterruptedException {
     boolean found = false;
     switch (type) {
-      case MASTER:
       case MANAGER:
         if (procRef.getProcess().equals(managerProcess)) {
           try {
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index 15096b1..d293b1b 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.miniclusterImpl;
 
+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.NANOSECONDS;
 import static java.util.stream.Collectors.toList;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -91,8 +91,8 @@
 import org.apache.accumulo.server.util.AccumuloStatus;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.start.Main;
-import org.apache.accumulo.start.classloader.vfs.MiniDFSUtil;
 import org.apache.accumulo.start.spi.KeywordExecutable;
+import org.apache.accumulo.start.util.MiniDFSUtil;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -111,7 +111,6 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Suppliers;
-import com.google.common.collect.Maps;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -154,7 +153,6 @@
   /**
    * @param config initial configuration
    */
-  @SuppressWarnings("deprecation")
   public MiniAccumuloClusterImpl(MiniAccumuloConfigImpl config) throws IOException {
 
     this.config = config.initialize();
@@ -225,13 +223,6 @@
     // written out and MAC started.
     config.preStartConfigUpdate();
 
-    File clientConfFile = config.getClientConfFile();
-    // Write only the properties that correspond to ClientConfiguration properties
-    writeConfigProperties(clientConfFile,
-        Maps.filterEntries(config.getSiteConfig(),
-            v -> org.apache.accumulo.core.client.ClientConfiguration.ClientProperty
-                .getPropertyByKey(v.getKey()) != null));
-
     Map<String,String> clientProps = config.getClientProps();
     clientProps.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), config.getZooKeepers());
     clientProps.put(ClientProperty.INSTANCE_NAME.getKey(), config.getInstanceName());
@@ -368,8 +359,6 @@
     builder.environment().put("CLASSPATH", classpath);
     builder.environment().put("ACCUMULO_HOME", config.getDir().getAbsolutePath());
     builder.environment().put("ACCUMULO_LOG_DIR", config.getLogDir().getAbsolutePath());
-    builder.environment().put("ACCUMULO_CLIENT_CONF_PATH",
-        config.getClientConfFile().getAbsolutePath());
     String ldLibraryPath = Joiner.on(File.pathSeparator).join(config.getNativeLibPaths());
     builder.environment().put("LD_LIBRARY_PATH", ldLibraryPath);
     builder.environment().put("DYLD_LIBRARY_PATH", ldLibraryPath);
@@ -864,13 +853,6 @@
     return Accumulo.newClient().from(clientProperties.get()).as(user, token).build();
   }
 
-  @SuppressWarnings("deprecation")
-  @Override
-  public org.apache.accumulo.core.client.ClientConfiguration getClientConfig() {
-    return org.apache.accumulo.core.client.ClientConfiguration.fromMap(config.getSiteConfig())
-        .withInstance(this.getInstanceName()).withZkHosts(this.getZooKeepers());
-  }
-
   @Override
   public Properties getClientProperties() {
     // return a copy, without re-reading the file
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index 025b320..b133280 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -160,24 +160,12 @@
       // since there is a small amount of memory, check more frequently for majc... setting may not
       // be needed in 1.5
       mergeProp(Property.TSERV_MAJC_DELAY.getKey(), "3");
-      @SuppressWarnings("deprecation")
-      Property generalClasspaths = Property.GENERAL_CLASSPATHS;
-      mergeProp(generalClasspaths.getKey(), libDir.getAbsolutePath() + "/[^.].*[.]jar");
-      @SuppressWarnings("deprecation")
-      Property generalDynamicClasspaths = Property.GENERAL_DYNAMIC_CLASSPATHS;
-      mergeProp(generalDynamicClasspaths.getKey(), libExtDir.getAbsolutePath() + "/[^.].*[.]jar");
       mergeProp(Property.GC_CYCLE_DELAY.getKey(), "4s");
       mergeProp(Property.GC_CYCLE_START.getKey(), "0s");
       mergePropWithRandomPort(Property.MANAGER_CLIENTPORT.getKey());
       mergePropWithRandomPort(Property.TSERV_CLIENTPORT.getKey());
       mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
       mergePropWithRandomPort(Property.GC_PORT.getKey());
-      @SuppressWarnings("deprecation")
-      Property p = Property.REPLICATION_RECEIPT_SERVICE_PORT;
-      mergePropWithRandomPort(p.getKey());
-      @SuppressWarnings("deprecation")
-      Property p2 = Property.MANAGER_REPLICATION_COORDINATOR_PORT;
-      mergePropWithRandomPort(p2.getKey());
 
       if (isUseCredentialProvider()) {
         updateConfigForCredentialProvider();
@@ -581,16 +569,6 @@
     this.useMiniDFS = useMiniDFS;
   }
 
-  /**
-   * @return location of client conf file containing connection parameters for connecting to this
-   *         minicluster
-   *
-   * @since 1.6.0
-   */
-  public File getClientConfFile() {
-    return new File(getConfDir(), "client.conf");
-  }
-
   public File getAccumuloPropsFile() {
     return new File(getConfDir(), "accumulo.properties");
   }
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterClasspathTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterClasspathTest.java
index d18ffdd..2a5da74 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterClasspathTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterClasspathTest.java
@@ -28,6 +28,8 @@
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -50,10 +52,6 @@
 @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
 public class MiniAccumuloClusterClasspathTest extends WithTestNames {
 
-  @SuppressWarnings("removal")
-  private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
-      Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-
   @TempDir
   private static File tempDir;
 
@@ -61,7 +59,7 @@
   public static final String ROOT_USER = "root";
 
   public static File testDir;
-
+  private static File jarFile;
   private static MiniAccumuloCluster accumulo;
 
   @BeforeAll
@@ -72,7 +70,7 @@
     FileUtils.deleteQuietly(testDir);
     assertTrue(testDir.mkdir());
 
-    File jarFile = new File(tempDir, "iterator.jar");
+    jarFile = new File(tempDir, "iterator.jar");
     FileUtils.copyURLToFile(
         requireNonNull(MiniAccumuloClusterClasspathTest.class.getResource("/FooFilter.jar")),
         jarFile);
@@ -80,7 +78,7 @@
     MiniAccumuloConfig config = new MiniAccumuloConfig(testDir, ROOT_PASSWORD).setJDWPEnabled(true);
     config.setZooKeeperPort(0);
     HashMap<String,String> site = new HashMap<>();
-    site.put(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1", jarFile.toURI().toString());
+    site.put(Property.TSERV_WAL_MAX_SIZE.getKey(), "1G");
     config.setSiteConfig(site);
     accumulo = new MiniAccumuloCluster(config);
     accumulo.start();
@@ -91,48 +89,50 @@
     accumulo.stop();
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   @Timeout(60)
   public void testPerTableClasspath() throws Exception {
-    org.apache.accumulo.core.client.Connector conn =
-        accumulo.getConnector(ROOT_USER, ROOT_PASSWORD);
+    try (AccumuloClient client = Accumulo.newClient().from(accumulo.getClientProperties())
+        .as(ROOT_USER, ROOT_PASSWORD).build()) {
 
-    final String tableName = testName();
+      final String tableName = testName();
 
-    var ntc = new NewTableConfiguration();
-    ntc.setProperties(Map.of(Property.TABLE_CLASSLOADER_CONTEXT.getKey(), "cx1"));
-    ntc.attachIterator(new IteratorSetting(100, "foocensor", "org.apache.accumulo.test.FooFilter"));
+      var ntc = new NewTableConfiguration();
+      ntc.setProperties(
+          Map.of(Property.TABLE_CLASSLOADER_CONTEXT.getKey(), jarFile.toURI().toString()));
+      ntc.attachIterator(
+          new IteratorSetting(100, "foocensor", "org.apache.accumulo.test.FooFilter"));
 
-    conn.tableOperations().create(tableName, ntc);
+      client.tableOperations().create(tableName, ntc);
 
-    try (BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig())) {
+      try (BatchWriter bw = client.createBatchWriter(tableName, new BatchWriterConfig())) {
 
-      Mutation m1 = new Mutation("foo");
-      m1.put("cf1", "cq1", "v2");
-      m1.put("cf1", "cq2", "v3");
+        Mutation m1 = new Mutation("foo");
+        m1.put("cf1", "cq1", "v2");
+        m1.put("cf1", "cq2", "v3");
 
-      bw.addMutation(m1);
+        bw.addMutation(m1);
 
-      Mutation m2 = new Mutation("bar");
-      m2.put("cf1", "cq1", "v6");
-      m2.put("cf1", "cq2", "v7");
+        Mutation m2 = new Mutation("bar");
+        m2.put("cf1", "cq1", "v6");
+        m2.put("cf1", "cq2", "v7");
 
-      bw.addMutation(m2);
+        bw.addMutation(m2);
 
-    }
-
-    int count = 0;
-    try (Scanner scanner = conn.createScanner(tableName, new Authorizations())) {
-      for (Entry<Key,Value> entry : scanner) {
-        assertFalse(entry.getKey().getRowData().toString().toLowerCase().contains("foo"));
-        count++;
       }
+
+      int count = 0;
+      try (Scanner scanner = client.createScanner(tableName, new Authorizations())) {
+        for (Entry<Key,Value> entry : scanner) {
+          assertFalse(entry.getKey().getRowData().toString().toLowerCase().contains("foo"));
+          count++;
+        }
+      }
+
+      assertEquals(2, count);
+
+      client.tableOperations().delete(tableName);
     }
-
-    assertEquals(2, count);
-
-    conn.tableOperations().delete(tableName);
   }
 
 }
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
index 8efedb5..fdff86e 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java
@@ -25,6 +25,8 @@
 import java.io.File;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.commons.io.FileUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -56,28 +58,29 @@
     config = new MiniAccumuloConfig(testDir, SECRET);
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void canConnectViaExistingZooKeeper() throws Exception {
     try (TestingServer zooKeeper = new TestingServer(); MiniAccumuloCluster accumulo =
         new MiniAccumuloCluster(config.setExistingZooKeepers(zooKeeper.getConnectString()))) {
       accumulo.start();
+      assertEquals(zooKeeper.getConnectString(), accumulo.getZooKeepers());
 
-      org.apache.accumulo.core.client.Connector conn = accumulo.getConnector("root", SECRET);
-      assertEquals(zooKeeper.getConnectString(), conn.getInstance().getZooKeepers());
+      try (AccumuloClient client =
+          Accumulo.newClient().from(accumulo.getClientProperties()).as("root", SECRET).build()) {
 
-      String tableName = "foo";
-      conn.tableOperations().create(tableName);
-      Map<String,String> tableIds = conn.tableOperations().tableIdMap();
-      assertTrue(tableIds.containsKey(tableName));
+        String tableName = "foo";
+        client.tableOperations().create(tableName);
+        Map<String,String> tableIds = client.tableOperations().tableIdMap();
+        assertTrue(tableIds.containsKey(tableName));
 
-      String zkTablePath = String.format("/accumulo/%s/tables/%s/name",
-          conn.getInstance().getInstanceID(), tableIds.get(tableName));
-      try (CuratorFramework client =
-          CuratorFrameworkFactory.newClient(zooKeeper.getConnectString(), new RetryOneTime(1))) {
-        client.start();
-        assertNotNull(client.checkExists().forPath(zkTablePath));
-        assertEquals(tableName, new String(client.getData().forPath(zkTablePath)));
+        String zkTablePath = String.format("/accumulo/%s/tables/%s/name",
+            client.instanceOperations().getInstanceId().canonical(), tableIds.get(tableName));
+        try (CuratorFramework curatorClient =
+            CuratorFrameworkFactory.newClient(zooKeeper.getConnectString(), new RetryOneTime(1))) {
+          curatorClient.start();
+          assertNotNull(curatorClient.checkExists().forPath(zkTablePath));
+          assertEquals(tableName, new String(curatorClient.getData().forPath(zkTablePath)));
+        }
       }
     }
   }
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterStartStopTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterStartStopTest.java
index 549bb50..5546547 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterStartStopTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterStartStopTest.java
@@ -23,6 +23,8 @@
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.commons.io.FileUtils;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
@@ -71,13 +73,14 @@
     }
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void multipleStopsIsAllowed() throws Exception {
     accumulo.start();
 
-    org.apache.accumulo.core.client.Connector conn = accumulo.getConnector("root", "superSecret");
-    conn.tableOperations().create("foo");
+    try (AccumuloClient client = Accumulo.newClient().from(accumulo.getClientProperties())
+        .as("root", "superSecret").build()) {
+      client.tableOperations().create("foo");
+    }
 
     accumulo.stop();
     accumulo.stop();
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
index 913904e..d4a48de 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
@@ -32,6 +32,8 @@
 import java.util.Set;
 import java.util.UUID;
 
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -79,7 +81,7 @@
     MiniAccumuloConfig config = new MiniAccumuloConfig(testDir, ROOT_PASSWORD).setJDWPEnabled(true);
     config.setZooKeeperPort(0);
     HashMap<String,String> site = new HashMap<>();
-    site.put(Property.TSERV_COMPACTION_WARN_TIME.getKey(), "5m");
+    site.put(Property.TSERV_WAL_SORT_BUFFER_SIZE.getKey(), "15%");
     config.setSiteConfig(site);
     accumulo = new MiniAccumuloCluster(config);
     accumulo.start();
@@ -94,88 +96,90 @@
     assertEquals("dfs.replication", DFSConfigKeys.DFS_REPLICATION_KEY);
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   @Timeout(30)
   public void test() throws Exception {
-    org.apache.accumulo.core.client.Connector conn =
-        accumulo.getConnector(ROOT_USER, ROOT_PASSWORD);
+    try (AccumuloClient conn = Accumulo.newClient().from(accumulo.getClientProperties())
+        .as(ROOT_USER, ROOT_PASSWORD).build()) {
 
-    final String tableName = testName();
+      final String tableName = testName();
 
-    IteratorSetting is = new IteratorSetting(10, SummingCombiner.class);
-    SummingCombiner.setEncodingType(is, LongCombiner.Type.STRING);
-    SummingCombiner.setColumns(is,
-        Collections.singletonList(new IteratorSetting.Column("META", "COUNT")));
+      IteratorSetting is = new IteratorSetting(10, SummingCombiner.class);
+      SummingCombiner.setEncodingType(is, LongCombiner.Type.STRING);
+      SummingCombiner.setColumns(is,
+          Collections.singletonList(new IteratorSetting.Column("META", "COUNT")));
 
-    conn.tableOperations().create(tableName, new NewTableConfiguration().attachIterator(is));
+      conn.tableOperations().create(tableName, new NewTableConfiguration().attachIterator(is));
 
-    final String principal = "user1";
-    final String password = "pass1";
-    conn.securityOperations().createLocalUser(principal, new PasswordToken(password));
-    conn.securityOperations().changeUserAuthorizations(principal, new Authorizations("A", "B"));
-    conn.securityOperations().grantTablePermission(principal, tableName, TablePermission.WRITE);
-    conn.securityOperations().grantTablePermission(principal, tableName, TablePermission.READ);
+      final String principal = "user1";
+      final String password = "pass1";
+      conn.securityOperations().createLocalUser(principal, new PasswordToken(password));
+      conn.securityOperations().changeUserAuthorizations(principal, new Authorizations("A", "B"));
+      conn.securityOperations().grantTablePermission(principal, tableName, TablePermission.WRITE);
+      conn.securityOperations().grantTablePermission(principal, tableName, TablePermission.READ);
 
-    org.apache.accumulo.core.client.Connector uconn = accumulo.getConnector(principal, password);
+      try (AccumuloClient uconn = Accumulo.newClient().from(accumulo.getClientProperties())
+          .as(principal, password).build()) {
 
-    try (BatchWriter bw = uconn.createBatchWriter(tableName, new BatchWriterConfig())) {
+        try (BatchWriter bw = uconn.createBatchWriter(tableName, new BatchWriterConfig())) {
 
-      UUID uuid = UUID.randomUUID();
+          UUID uuid = UUID.randomUUID();
 
-      ColumnVisibility colVisAorB = new ColumnVisibility("A|B");
-      Mutation m = new Mutation(uuid.toString());
-      m.put("META", "SIZE", colVisAorB, "8");
-      m.put("META", "CRC", colVisAorB, "456");
-      m.put("META", "COUNT", colVisAorB, "1");
-      m.put("DATA", "IMG", new ColumnVisibility("A&B"), "ABCDEFGH");
+          ColumnVisibility colVisAorB = new ColumnVisibility("A|B");
+          Mutation m = new Mutation(uuid.toString());
+          m.put("META", "SIZE", colVisAorB, "8");
+          m.put("META", "CRC", colVisAorB, "456");
+          m.put("META", "COUNT", colVisAorB, "1");
+          m.put("DATA", "IMG", new ColumnVisibility("A&B"), "ABCDEFGH");
 
-      bw.addMutation(m);
-      bw.flush();
+          bw.addMutation(m);
+          bw.flush();
 
-      m = new Mutation(uuid.toString());
-      m.put("META", "COUNT", colVisAorB, "1");
-      m.put("META", "CRC", colVisAorB, "123");
-      bw.addMutation(m);
+          m = new Mutation(uuid.toString());
+          m.put("META", "COUNT", colVisAorB, "1");
+          m.put("META", "CRC", colVisAorB, "123");
+          bw.addMutation(m);
 
-    }
-
-    int count = 0;
-    try (Scanner scanner = uconn.createScanner(tableName, new Authorizations("A"))) {
-      for (Entry<Key,Value> entry : scanner) {
-        final String actualValue = entry.getValue().toString();
-        switch (entry.getKey().getColumnQualifierData().toString()) {
-          case "COUNT":
-            assertEquals("2", actualValue);
-            break;
-          case "SIZE":
-            assertEquals("8", actualValue);
-            break;
-          case "CRC":
-            assertEquals("123", actualValue);
-            break;
-          default:
-            fail();
-            break;
         }
-        count++;
-      }
-    }
-    assertEquals(3, count);
 
-    count = 0;
-    try (Scanner scanner = uconn.createScanner(tableName, new Authorizations("A", "B"))) {
-      for (Entry<Key,Value> entry : scanner) {
-        if (entry.getKey().getColumnQualifierData().toString().equals("IMG")) {
-          assertEquals("ABCDEFGH", entry.getValue().toString());
+        int count = 0;
+        try (Scanner scanner = uconn.createScanner(tableName, new Authorizations("A"))) {
+          for (Entry<Key,Value> entry : scanner) {
+            final String actualValue = entry.getValue().toString();
+            switch (entry.getKey().getColumnQualifierData().toString()) {
+              case "COUNT":
+                assertEquals("2", actualValue);
+                break;
+              case "SIZE":
+                assertEquals("8", actualValue);
+                break;
+              case "CRC":
+                assertEquals("123", actualValue);
+                break;
+              default:
+                fail();
+                break;
+            }
+            count++;
+          }
         }
-        count++;
+        assertEquals(3, count);
+
+        count = 0;
+        try (Scanner scanner = uconn.createScanner(tableName, new Authorizations("A", "B"))) {
+          for (Entry<Key,Value> entry : scanner) {
+            if (entry.getKey().getColumnQualifierData().toString().equals("IMG")) {
+              assertEquals("ABCDEFGH", entry.getValue().toString());
+            }
+            count++;
+          }
+        }
+
+        assertEquals(4, count);
       }
+      conn.tableOperations().delete(tableName);
     }
 
-    assertEquals(4, count);
-
-    conn.tableOperations().delete(tableName);
   }
 
   @Test
@@ -194,7 +198,7 @@
     // ensure what user passed in is what comes back
     assertEquals(0, accumulo.getConfig().getZooKeeperPort());
     HashMap<String,String> site = new HashMap<>();
-    site.put(Property.TSERV_COMPACTION_WARN_TIME.getKey(), "5m");
+    site.put(Property.TSERV_WAL_SORT_BUFFER_SIZE.getKey(), "15%");
     assertEquals(site, accumulo.getConfig().getSiteConfig());
   }
 
diff --git a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
index 5ec9e68..c8cf747 100644
--- a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
@@ -35,8 +35,7 @@
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
-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.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.commons.io.FileUtils;
@@ -107,9 +106,10 @@
   @Timeout(60)
   public void saneMonitorInfo() throws Exception {
     ManagerMonitorInfo stats;
+    int expectedNumTables = 3;
     while (true) {
       stats = accumulo.getManagerMonitorInfo();
-      if (stats.tableMap.size() <= 2) {
+      if (stats.tableMap.size() < expectedNumTables) {
         continue;
       }
 
@@ -123,9 +123,9 @@
     assertTrue(validGoals.contains(stats.goalState),
         "manager goal state should be in " + validGoals + ". is " + stats.goalState);
     assertNotNull(stats.tableMap, "should have a table map.");
-    assertTrue(stats.tableMap.containsKey(RootTable.ID.canonical()),
+    assertTrue(stats.tableMap.containsKey(AccumuloTable.ROOT.tableId().canonical()),
         "root table should exist in " + stats.tableMap.keySet());
-    assertTrue(stats.tableMap.containsKey(MetadataTable.ID.canonical()),
+    assertTrue(stats.tableMap.containsKey(AccumuloTable.METADATA.tableId().canonical()),
         "meta table should exist in " + stats.tableMap.keySet());
     assertTrue(stats.tableMap.containsKey(testTableID),
         "our test table should exist in " + stats.tableMap.keySet());
diff --git a/pom.xml b/pom.xml
index 463fd29..901e9db 100644
--- a/pom.xml
+++ b/pom.xml
@@ -28,7 +28,7 @@
   </parent>
   <groupId>org.apache.accumulo</groupId>
   <artifactId>accumulo-project</artifactId>
-  <version>2.1.3-SNAPSHOT</version>
+  <version>3.1.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Accumulo Project</name>
   <description>Apache Accumulo is a sorted, distributed key/value store based
@@ -89,7 +89,6 @@
     <module>server/compaction-coordinator</module>
     <module>server/gc</module>
     <module>server/manager</module>
-    <module>server/master</module>
     <module>server/monitor</module>
     <module>server/native</module>
     <module>server/tserver</module>
@@ -136,7 +135,7 @@
     <maven.version.ignore>.+-SNAPSHOT,(?i).*(alpha|beta)[0-9.-]*,(?i).*[.-](m|rc)[0-9]+</maven.version.ignore>
     <minimalJavaBuildVersion>17</minimalJavaBuildVersion>
     <!-- timestamp for reproducible outputs, updated on release by the release plugin -->
-    <project.build.outputTimestamp>2023-08-14T08:01:47Z</project.build.outputTimestamp>
+    <project.build.outputTimestamp>2023-08-14T08:11:10Z</project.build.outputTimestamp>
     <rat.consoleOutput>true</rat.consoleOutput>
     <sourceReleaseAssemblyDescriptor>source-release-tar</sourceReleaseAssemblyDescriptor>
     <surefire.excludedGroups />
@@ -291,11 +290,6 @@
         <version>33.0.0-jre</version>
       </dependency>
       <dependency>
-        <groupId>com.google.protobuf</groupId>
-        <artifactId>protobuf-java</artifactId>
-        <version>3.22.0</version>
-      </dependency>
-      <dependency>
         <groupId>com.lmax</groupId>
         <artifactId>disruptor</artifactId>
         <!-- log4j doesn't support 4 yet; see https://github.com/apache/logging-log4j2/issues/1829 -->
@@ -322,12 +316,6 @@
         <version>1.2</version>
       </dependency>
       <dependency>
-        <!-- legacy junit version specified here for dependency convergence -->
-        <groupId>junit</groupId>
-        <artifactId>junit</artifactId>
-        <version>4.13.2</version>
-      </dependency>
-      <dependency>
         <groupId>org.apache.accumulo</groupId>
         <artifactId>accumulo-compaction-coordinator</artifactId>
         <version>${project.version}</version>
@@ -444,17 +432,6 @@
         <version>1.11.0</version>
       </dependency>
       <dependency>
-        <groupId>org.apache.commons</groupId>
-        <artifactId>commons-vfs2</artifactId>
-        <version>2.9.0</version>
-        <exclusions>
-          <exclusion>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-hdfs-client</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
         <groupId>org.apache.curator</groupId>
         <artifactId>curator-framework</artifactId>
         <version>${version.curator}</version>
@@ -638,32 +615,6 @@
         </exclusions>
       </dependency>
       <dependency>
-        <!-- converge transitive dependency version between powermock and easymock -->
-        <groupId>org.objenesis</groupId>
-        <artifactId>objenesis</artifactId>
-        <version>3.3</version>
-      </dependency>
-      <dependency>
-        <groupId>org.powermock</groupId>
-        <artifactId>powermock-api-easymock</artifactId>
-        <version>${version.powermock}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.powermock</groupId>
-        <artifactId>powermock-core</artifactId>
-        <version>${version.powermock}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.powermock</groupId>
-        <artifactId>powermock-module-junit4</artifactId>
-        <version>${version.powermock}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.powermock</groupId>
-        <artifactId>powermock-reflect</artifactId>
-        <version>${version.powermock}</version>
-      </dependency>
-      <dependency>
         <groupId>org.slf4j</groupId>
         <artifactId>slf4j-api</artifactId>
         <version>${version.slf4j}</version>
@@ -710,6 +661,7 @@
                   <exclude>**/LICENSE</exclude>
                   <exclude>**/NOTICE</exclude>
                   <exclude>**/target/**</exclude>
+                  <exclude>**/*.rf</exclude>
                 </excludes>
               </licenseSet>
             </licenseSets>
@@ -721,7 +673,6 @@
               <css>SLASHSTAR_STYLE</css>
               <h>SLASHSTAR_STYLE</h>
               <java>SLASHSTAR_STYLE</java>
-              <proto>SLASHSTAR_STYLE</proto>
               <thrift>SLASHSTAR_STYLE</thrift>
             </mapping>
           </configuration>
@@ -808,8 +759,7 @@
           <configuration>
             <archive>
               <manifestEntries>
-                <!-- Automatic modules do not work with the javadoc plugin - see MJAVADOC-707 -->
-                <!-- Automatic-Module-Name>${accumulo.module.name}</Automatic-Module-Name -->
+                <Automatic-Module-Name>${accumulo.module.name}</Automatic-Module-Name>
                 <Implementation-Build>${mvngit.commit.id}</Implementation-Build>
                 <Sealed>true</Sealed>
               </manifestEntries>
@@ -826,6 +776,7 @@
             <quiet>true</quiet>
             <additionalJOption>-J-Xmx512m</additionalJOption>
             <doclint>all,-missing</doclint>
+            <legacyMode>true</legacyMode>
           </configuration>
         </plugin>
         <plugin>
@@ -895,7 +846,6 @@
             <configFile>${rootlocation}/src/build/eclipse-codestyle.xml</configFile>
             <excludes>
               <exclude>**/thrift/*.java</exclude>
-              <exclude>**/proto/*.java</exclude>
             </excludes>
             <lineEnding>LF</lineEnding>
             <skipCssFormatting>true</skipCssFormatting>
@@ -917,6 +867,7 @@
               <exclude>.vscode/**</exclude>
               <exclude>.factorypath</exclude>
               <exclude>.github/**</exclude>
+              <exclude>**/*.rf</exclude>
             </excludes>
           </configuration>
         </plugin>
@@ -963,8 +914,6 @@
                 <undeclared>org.apache.httpcomponents:httpcore:jar:*</undeclared>
                 <undeclared>org.powermock:powermock-core:jar:*</undeclared>
                 <undeclared>org.powermock:powermock-reflect:jar:*</undeclared>
-                <!-- This should be removed upon completion of migrating junit 4 to 5 -->
-                <undeclared>junit:junit:jar:*</undeclared>
               </ignoredUsedUndeclaredDependencies>
               <ignoredUnusedDeclaredDependencies>
                 <!-- auto-service isn't used in every module -->
@@ -988,8 +937,6 @@
                 <unused>org.apache.logging.log4j:log4j-web:jar:*</unused>
                 <!-- ignore log4j dep used for annotations and needed for compile time linting -->
                 <unused>biz.aQute.bnd:biz.aQute.bnd.annotation:jar:*</unused>
-                <!-- This should be removed upon completion of migrating junit 4 to 5 -->
-                <unused>org.junit.vintage:junit-vintage-engine:jar:*</unused>
                 <unused>org.junit.jupiter:junit-jupiter-engine:jar:*</unused>
               </ignoredUnusedDeclaredDependencies>
             </configuration>
@@ -1088,6 +1035,10 @@
                   <property name="message" value="Line has trailing whitespace." />
                 </module>
                 <module name="RegexpSinglelineJava">
+                  <property name="format" value="[@]Deprecated([^)]*forRemoval[^)]*)" />
+                  <property name="message" value="forRemoval should not be used." />
+                </module>
+                <module name="RegexpSinglelineJava">
                   <property name="format" value="[@]see\s+[{][@]link" />
                   <property name="message" value="Javadoc @see does not need @link: pick one or the other." />
                 </module>
@@ -1100,22 +1051,14 @@
                   <property name="message" value="Use commons-math3 (org.apache.commons.math3.*)" />
                 </module>
                 <module name="RegexpSinglelineJava">
-                  <property name="format" value="junit[.]framework[.]TestCase" />
-                  <property name="message" value="Use JUnit4+ @Test annotation instead of TestCase" />
-                </module>
-                <module name="RegexpSinglelineJava">
-                  <property name="format" value="import org[.]junit[.]Assert;" />
-                  <property name="message" value="Use static imports for Assert.* methods for consistency" />
+                  <property name="format" value="import org[.]apache[.]accumulo[.]core[.]util[.]LazySingletons;" />
+                  <property name="message" value="Use static imports for LazySingletons for consistency" />
                 </module>
                 <module name="RegexpSinglelineJava">
                   <property name="format" value="org[.]junit[.]jupiter[.]api[.]Assertions;" />
                   <property name="message" value="Use static imports for Assertions.* methods for consistency" />
                 </module>
                 <module name="RegexpSinglelineJava">
-                  <property name="format" value="import org[.]junit[.]Assume;" />
-                  <property name="message" value="Use static imports for Assume.* methods for consistency" />
-                </module>
-                <module name="RegexpSinglelineJava">
                   <property name="format" value="org[.]junit[.]jupiter[.]api[.]Assumptions;" />
                   <property name="message" value="Use static imports for Assumptions.* methods for consistency" />
                 </module>
@@ -1128,6 +1071,10 @@
                   <property name="format" value="&amp;quot; [+] &amp;quot;" />
                   <property name="message" value="Unnecessary concatenation of string literals" />
                 </module>
+                <module name="RegexpSinglelineJava">
+                  <property name="format" value="com[.]google[.]common[.]cache[.]" />
+                  <property name="message" value="Please use Caffeine Cache, not Guava" />
+                </module>
                 <module name="OuterTypeFilename" />
                 <module name="AvoidStarImport" />
                 <module name="NoLineWrap" />
@@ -1163,7 +1110,7 @@
           </checkstyleRules>
           <violationSeverity>warning</violationSeverity>
           <includeTestSourceDirectory>true</includeTestSourceDirectory>
-          <excludes>**/thrift/*.java,**/proto/*.java</excludes>
+          <excludes>**/thrift/*.java</excludes>
         </configuration>
         <dependencies>
           <dependency>
@@ -1665,7 +1612,7 @@
                 <arg>-XDcompilePolicy=simple</arg>
                 <arg>
                   -Xplugin:ErrorProne \
-                  -XepExcludedPaths:.*/(proto|thrift|generated-sources|src/test)/.* \
+                  -XepExcludedPaths:.*/(thrift|generated-sources|src/test)/.* \
                   -XepDisableWarningsInGeneratedCode \
                   -XepDisableAllWarnings \
                   <!-- ERROR patterns to specifically ignore -->
diff --git a/server/base/pom.xml b/server/base/pom.xml
index 1398428..6b76774 100644
--- a/server/base/pom.xml
+++ b/server/base/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-server-base</artifactId>
@@ -53,10 +53,6 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
@@ -143,29 +139,6 @@
   </build>
   <profiles>
     <profile>
-      <id>protobuf</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.codehaus.mojo</groupId>
-            <artifactId>exec-maven-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>generate-protobuf</id>
-                <goals>
-                  <goal>exec</goal>
-                </goals>
-                <phase>generate-sources</phase>
-                <configuration>
-                  <executable>${basedir}/src/main/scripts/generate-protobuf.sh</executable>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <profile>
       <!-- profile for running second execution of surefire without reusing forks -->
       <!-- on by default, but disable with '-P !reuseForksFalse' or '-Dtest=SomeTest' -->
       <id>reuseForksFalse</id>
diff --git a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
index ac1bcaa..bd66689 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
@@ -18,32 +18,42 @@
  */
 package org.apache.accumulo.server;
 
-import java.util.Objects;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metrics.MetricsProducer;
 import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.server.mem.LowMemoryDetector;
+import org.apache.accumulo.server.metrics.ProcessMetrics;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class AbstractServer implements AutoCloseable, Runnable {
+import io.micrometer.core.instrument.MeterRegistry;
+
+public abstract class AbstractServer implements AutoCloseable, MetricsProducer, Runnable {
 
   private final ServerContext context;
   protected final String applicationName;
   private final String hostname;
-  private final Logger log;
 
-  protected AbstractServer(String appName, ServerOpts opts, String[] args) {
-    this.log = LoggerFactory.getLogger(getClass().getName());
+  private final ProcessMetrics processMetrics;
+
+  protected AbstractServer(String appName, ConfigOpts opts, String[] args) {
     this.applicationName = appName;
     opts.parseArgs(appName, args);
-    this.hostname = Objects.requireNonNull(opts.getAddress());
     var siteConfig = opts.getSiteConfiguration();
+    this.hostname = siteConfig.get(Property.GENERAL_PROCESS_BIND_ADDRESS);
     SecurityUtil.serverLogin(siteConfig);
     context = new ServerContext(siteConfig);
+    Logger log = LoggerFactory.getLogger(getClass());
     log.info("Version " + Constants.VERSION);
     log.info("Instance " + context.getInstanceID());
     context.init(appName);
@@ -53,6 +63,12 @@
       // Server-side "client" check to make sure we're logged in as a user we expect to be
       context.enforceKerberosLogin();
     }
+    final LowMemoryDetector lmd = context.getLowMemoryDetector();
+    ScheduledFuture<?> future = context.getScheduledExecutor().scheduleWithFixedDelay(
+        () -> lmd.logGCInfo(context.getConfiguration()), 0,
+        lmd.getIntervalMillis(context.getConfiguration()), TimeUnit.MILLISECONDS);
+    ThreadPools.watchNonCriticalScheduledTask(future);
+    processMetrics = new ProcessMetrics(context);
   }
 
   /**
@@ -72,7 +88,18 @@
       if (thrown instanceof Exception) {
         throw (Exception) thrown;
       }
-      throw new RuntimeException("Weird throwable type thrown", thrown);
+      throw new IllegalStateException("Weird throwable type thrown", thrown);
+    }
+  }
+
+  /**
+   * Called
+   */
+  @Override
+  public void registerMetrics(MeterRegistry registry) {
+    // makes mocking subclasses easier
+    if (processMetrics != null) {
+      processMetrics.registerMetrics(registry);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/AccumuloDataVersion.java b/server/base/src/main/java/org/apache/accumulo/server/AccumuloDataVersion.java
index 666f113..5cbb0e2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/AccumuloDataVersion.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/AccumuloDataVersion.java
@@ -22,49 +22,50 @@
 
 /**
  * Class representing the version of data stored in Accumulo.
- *
+ * <p>
  * This version is separate but related to the file specific version in
  * {@link org.apache.accumulo.core.file.rfile.RFile}. A version change to RFile will reflect a
  * version change to the AccumuloDataVersion. But a version change to the AccumuloDataVersion may
  * not affect the version number in RFile. For example, changes made to other parts of Accumulo that
  * affects how data is stored, like the metadata table, would change the AccumuloDataVersion number
  * here but not in RFile.
- *
+ * <p>
  * This number is stored in HDFS under {@link org.apache.accumulo.core.Constants#VERSION_DIR}.
- *
+ * <p>
  * This class is used for checking the version during server startup and upgrades.
  */
 public class AccumuloDataVersion {
 
   /**
+   * version (12) reflect changes to support no chop merges including json encoding of the file
+   * column family stored in root and metadata tables in version 3.1
+   */
+  public static final int METADATA_FILE_JSON_ENCODING = 12;
+
+  /**
+   * version (11) reflects removal of replication starting with 3.0
+   */
+  public static final int REMOVE_DEPRECATIONS_FOR_VERSION_3 = 11;
+
+  /**
    * version (10) reflects changes to how root tablet metadata is serialized in zookeeper starting
-   * with 2.1. See {@link org.apache.accumulo.core.metadata.schema.RootTabletMetadata}.
+   * with 2.1. See {@link org.apache.accumulo.core.metadata.schema.RootTabletMetadata}
    */
   public static final int ROOT_TABLET_META_CHANGES = 10;
 
   /**
-   * version (9) reflects changes to crypto that resulted in RFiles and WALs being serialized
-   * differently in version 2.0.0. Also RFiles in 2.0.0 may have summary data.
-   */
-  public static final int CRYPTO_CHANGES = 9;
-
-  /**
-   * version (8) reflects changes to RFile index (ACCUMULO-1124) AND the change to WAL tracking in
-   * ZK in version 1.8.0
-   */
-  public static final int SHORTEN_RFILE_KEYS = 8;
-
-  /**
-   * Historic data versions
+   * Historic data versions.
    *
    * <ul>
+   * <li>version (9) RFiles and wal crypto serialization changes. RFile summary data in 2.0.0</li>
+   * <li>version (8) RFile index (ACCUMULO-1124) and wal tracking in ZK in 1.8.0</li>
    * <li>version (7) also reflects the addition of a replication table in 1.7.0
    * <li>version (6) reflects the addition of a separate root table (ACCUMULO-1481) in 1.6.0 -
    * <li>version (5) moves delete file markers for the metadata table into the root tablet
    * <li>version (4) moves logging to HDFS in 1.5.0
    * </ul>
    */
-  private static final int CURRENT_VERSION = ROOT_TABLET_META_CHANGES;
+  private static final int CURRENT_VERSION = METADATA_FILE_JSON_ENCODING;
 
   /**
    * Get the current Accumulo Data Version. See Javadoc of static final integers for a detailed
@@ -77,5 +78,38 @@
   }
 
   public static final Set<Integer> CAN_RUN =
-      Set.of(SHORTEN_RFILE_KEYS, CRYPTO_CHANGES, CURRENT_VERSION);
+      Set.of(ROOT_TABLET_META_CHANGES, REMOVE_DEPRECATIONS_FOR_VERSION_3, CURRENT_VERSION);
+
+  /**
+   * Get the stored, current working version.
+   *
+   * @param context the server context
+   * @return the stored data version
+   */
+  public static int getCurrentVersion(ServerContext context) {
+    int cv =
+        context.getServerDirs().getAccumuloPersistentVersion(context.getVolumeManager().getFirst());
+    ServerContext.ensureDataVersionCompatible(cv);
+    return cv;
+  }
+
+  public static int oldestUpgradeableVersion() {
+    return CAN_RUN.stream().mapToInt(x -> x).min().orElseThrow();
+  }
+
+  public static String oldestUpgradeableVersionName() {
+    return dataVersionToReleaseName(oldestUpgradeableVersion());
+  }
+
+  private static String dataVersionToReleaseName(final int version) {
+    switch (version) {
+      case ROOT_TABLET_META_CHANGES:
+        return "2.1.0";
+      case REMOVE_DEPRECATIONS_FOR_VERSION_3:
+        return "3.0.0";
+      case METADATA_FILE_JSON_ENCODING:
+        return "3.1.0";
+    }
+    throw new IllegalArgumentException("Unsupported data version " + version);
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java b/server/base/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
deleted file mode 100644
index 469ec56..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
+++ /dev/null
@@ -1,130 +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.server;
-
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.Halt;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class GarbageCollectionLogger {
-  private static final Logger log = LoggerFactory.getLogger(GarbageCollectionLogger.class);
-
-  private final HashMap<String,Long> prevGcTime = new HashMap<>();
-  private long lastMemorySize = 0;
-  private long gcTimeIncreasedCount = 0;
-  private static long lastMemoryCheckTime = 0;
-  private static final Lock memCheckTimeLock = new ReentrantLock();
-
-  public void logGCInfo(AccumuloConfiguration conf) {
-
-    memCheckTimeLock.lock();
-    try {
-      final long now = System.currentTimeMillis();
-
-      List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
-      Runtime rt = Runtime.getRuntime();
-
-      StringBuilder sb = new StringBuilder("gc");
-
-      boolean sawChange = false;
-
-      long maxIncreaseInCollectionTime = 0;
-
-      for (GarbageCollectorMXBean gcBean : gcmBeans) {
-        Long prevTime = prevGcTime.get(gcBean.getName());
-        long pt = 0;
-        if (prevTime != null) {
-          pt = prevTime;
-        }
-
-        long time = gcBean.getCollectionTime();
-
-        if (time - pt != 0) {
-          sawChange = true;
-        }
-
-        long increaseInCollectionTime = time - pt;
-        sb.append(String.format(" %s=%,.2f(+%,.2f) secs", gcBean.getName(), time / 1000.0,
-            increaseInCollectionTime / 1000.0));
-        maxIncreaseInCollectionTime =
-            Math.max(increaseInCollectionTime, maxIncreaseInCollectionTime);
-        prevGcTime.put(gcBean.getName(), time);
-      }
-
-      long mem = rt.freeMemory();
-      if (maxIncreaseInCollectionTime == 0) {
-        gcTimeIncreasedCount = 0;
-      } else {
-        gcTimeIncreasedCount++;
-        if (gcTimeIncreasedCount > 3 && mem < rt.maxMemory() * 0.05) {
-          log.warn("Running low on memory");
-          gcTimeIncreasedCount = 0;
-        }
-      }
-
-      if (mem != lastMemorySize) {
-        sawChange = true;
-      }
-
-      String sign = "+";
-      if (mem - lastMemorySize <= 0) {
-        sign = "";
-      }
-
-      sb.append(String.format(" freemem=%,d(%s%,d) totalmem=%,d", mem, sign, (mem - lastMemorySize),
-          rt.totalMemory()));
-
-      if (sawChange) {
-        log.debug(sb.toString());
-      }
-
-      final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-      if (lastMemoryCheckTime > 0 && lastMemoryCheckTime < now) {
-        final long diff = now - lastMemoryCheckTime;
-        if (diff > keepAliveTimeout + 1000) {
-          log.warn(String.format(
-              "GC pause checker not called in a timely"
-                  + " fashion. Expected every %.1f seconds but was %.1f seconds since last check",
-              keepAliveTimeout / 1000., diff / 1000.));
-        }
-        lastMemoryCheckTime = now;
-        return;
-      }
-
-      if (maxIncreaseInCollectionTime > keepAliveTimeout) {
-        Halt.halt("Garbage collection may be interfering with lock keep-alive.  Halting.", -1);
-      }
-
-      lastMemorySize = mem;
-      lastMemoryCheckTime = now;
-    } finally {
-      memCheckTimeLock.unlock();
-    }
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 4366967..bf75821 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -20,18 +20,18 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Suppliers.memoize;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.UncheckedIOException;
 import java.net.UnknownHostException;
 import java.util.Arrays;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
@@ -58,7 +58,6 @@
 import org.apache.accumulo.core.singletons.SingletonReservation;
 import org.apache.accumulo.core.spi.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.conf.NamespaceConfiguration;
@@ -67,6 +66,7 @@
 import org.apache.accumulo.server.conf.store.PropStore;
 import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.mem.LowMemoryDetector;
 import org.apache.accumulo.server.metadata.ServerAmpleImpl;
 import org.apache.accumulo.server.metrics.MetricsInfoImpl;
 import org.apache.accumulo.server.rpc.SaslServerConnectionParams;
@@ -94,6 +94,7 @@
   private final ZooReaderWriter zooReaderWriter;
   private final ServerDirs serverDirs;
   private final Supplier<ZooPropStore> propStore;
+  private final Supplier<String> zkUserPath;
 
   // lazily loaded resources, only loaded when needed
   private final Supplier<TableManager> tableManager;
@@ -103,6 +104,7 @@
   private final Supplier<ScheduledThreadPoolExecutor> sharedScheduledThreadPool;
   private final Supplier<AuditedSecurityOperation> securityOperation;
   private final Supplier<CryptoServiceFactory> cryptoFactorySupplier;
+  private final Supplier<LowMemoryDetector> lowMemoryDetector;
   private final Supplier<MetricsInfo> metricsInfoSupplier;
 
   public ServerContext(SiteConfiguration siteConfig) {
@@ -116,6 +118,7 @@
     serverDirs = info.getServerDirs();
 
     propStore = memoize(() -> ZooPropStore.initialize(getInstanceID(), getZooReaderWriter()));
+    zkUserPath = memoize(() -> Constants.ZROOT + "/" + getInstanceID() + Constants.ZUSERS);
 
     tableManager = memoize(() -> new TableManager(this));
     nameAllocator = memoize(() -> new UniqueNameAllocator(this));
@@ -128,6 +131,7 @@
     securityOperation =
         memoize(() -> new AuditedSecurityOperation(this, SecurityOperation.getAuthorizor(this),
             SecurityOperation.getAuthenticator(this), SecurityOperation.getPermHandler(this)));
+    lowMemoryDetector = memoize(() -> new LowMemoryDetector());
     metricsInfoSupplier = memoize(() -> new MetricsInfoImpl(this));
   }
 
@@ -194,7 +198,7 @@
       // currentUser() like KerberosToken
       loginUser = UserGroupInformation.getLoginUser();
     } catch (IOException e) {
-      throw new RuntimeException("Could not get login user", e);
+      throw new UncheckedIOException("Could not get login user", e);
     }
 
     checkArgument(loginUser.hasKerberosCredentials(), "Server does not have Kerberos credentials");
@@ -284,7 +288,7 @@
     return serverDirs.getBaseUris();
   }
 
-  public List<Pair<Path,Path>> getVolumeReplacements() {
+  public Map<Path,Path> getVolumeReplacements() {
     return serverDirs.getVolumeReplacements();
   }
 
@@ -302,7 +306,9 @@
   public static void ensureDataVersionCompatible(int dataVersion) {
     if (!AccumuloDataVersion.CAN_RUN.contains(dataVersion)) {
       throw new IllegalStateException("This version of accumulo (" + Constants.VERSION
-          + ") is not compatible with files stored using data version " + dataVersion);
+          + ") is not compatible with files stored using data version " + dataVersion
+          + ". Please upgrade from " + AccumuloDataVersion.oldestUpgradeableVersionName()
+          + " or later.");
     }
   }
 
@@ -449,6 +455,14 @@
     return securityOperation.get();
   }
 
+  public String zkUserPath() {
+    return zkUserPath.get();
+  }
+
+  public LowMemoryDetector getLowMemoryDetector() {
+    return lowMemoryDetector.get();
+  }
+
   public MetricsInfo getMetricsInfo() {
     return metricsInfoSupplier.get();
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerDirs.java b/server/base/src/main/java/org/apache/accumulo/server/ServerDirs.java
index fd3f27a..062198b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerDirs.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerDirs.java
@@ -19,13 +19,14 @@
 package org.apache.accumulo.server;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
-import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
@@ -34,7 +35,6 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -54,14 +54,14 @@
   private Set<String> tablesDirs;
   private Set<String> recoveryDirs;
 
-  private final List<Pair<Path,Path>> replacementsList;
+  private final Map<Path,Path> replacements;
   private final AccumuloConfiguration conf;
   private final Configuration hadoopConf;
 
   public ServerDirs(AccumuloConfiguration conf, Configuration hadoopConf) {
     this.conf = Objects.requireNonNull(conf, "Configuration cannot be null");
     this.hadoopConf = Objects.requireNonNull(hadoopConf, "Hadoop configuration cannot be null");
-    this.replacementsList = loadVolumeReplacements();
+    this.replacements = loadVolumeReplacements();
   }
 
   public Set<String> getBaseUris() {
@@ -116,7 +116,7 @@
     }
 
     if (baseDirsList.isEmpty()) {
-      throw new RuntimeException("None of the configured paths are initialized.");
+      throw new IllegalStateException("None of the configured paths are initialized.");
     }
 
     return baseDirsList;
@@ -142,18 +142,18 @@
     return recoveryDirs;
   }
 
-  private List<Pair<Path,Path>> loadVolumeReplacements() {
+  private Map<Path,Path> loadVolumeReplacements() {
 
-    List<Pair<Path,Path>> replacementsList;
+    Map<Path,Path> replacementsMap;
     String replacements = conf.get(Property.INSTANCE_VOLUMES_REPLACEMENTS);
 
     if (replacements == null || replacements.trim().isEmpty()) {
-      return Collections.emptyList();
+      return Collections.emptyMap();
     }
     replacements = replacements.trim();
 
     String[] pairs = replacements.split(",");
-    List<Pair<Path,Path>> ret = new ArrayList<>();
+    Map<Path,Path> ret = new LinkedHashMap<>();
 
     for (String pair : pairs) {
 
@@ -187,7 +187,7 @@
             + " contains " + uris[1] + " which has a syntax error", e);
       }
 
-      ret.add(new Pair<>(p1, p2));
+      ret.put(p1, p2);
     }
 
     HashSet<Path> baseDirs = new HashSet<>();
@@ -196,21 +196,21 @@
       baseDirs.add(new Path(baseDir));
     }
 
-    for (Pair<Path,Path> pair : ret) {
-      if (!baseDirs.contains(pair.getSecond())) {
+    for (Path replacement : ret.values()) {
+      if (!baseDirs.contains(replacement)) {
         throw new IllegalArgumentException(Property.INSTANCE_VOLUMES_REPLACEMENTS.getKey()
-            + " contains " + pair.getSecond() + " which is not a configured volume");
+            + " contains " + replacement + " which is not a configured volume");
       }
     }
 
     // only set if get here w/o exception
-    replacementsList = ret;
+    replacementsMap = ret;
 
-    return replacementsList;
+    return replacementsMap;
   }
 
-  public List<Pair<Path,Path>> getVolumeReplacements() {
-    return this.replacementsList;
+  public Map<Path,Path> getVolumeReplacements() {
+    return this.replacements;
   }
 
   public Path getDataVersionLocation(Volume v) {
@@ -234,7 +234,7 @@
       }
       return dataVersion;
     } catch (IOException e) {
-      throw new RuntimeException("Unable to read accumulo version: an error occurred.", e);
+      throw new UncheckedIOException("Unable to read accumulo version: an error occurred.", e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java b/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
index d9eccd6..f0fd0c8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
@@ -21,6 +21,7 @@
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
@@ -67,21 +68,23 @@
     try {
       volumeManager = VolumeManagerImpl.get(siteConfig, hadoopConf);
     } catch (IOException e) {
-      throw new IllegalStateException(e);
+      throw new UncheckedIOException(e);
     }
     zooCache = new ZooCacheFactory().getZooCache(zooKeepers, zooKeepersSessionTimeOut);
     String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
     byte[] iidb = zooCache.get(instanceNamePath);
     if (iidb == null) {
-      throw new RuntimeException("Instance name " + instanceName + " does not exist in zookeeper. "
-          + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
+      throw new IllegalStateException(
+          "Instance name " + instanceName + " does not exist in zookeeper. "
+              + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
     }
     instanceID = InstanceId.of(new String(iidb, UTF_8));
     if (zooCache.get(Constants.ZROOT + "/" + instanceID) == null) {
       if (instanceName == null) {
-        throw new RuntimeException("Instance id " + instanceID + " does not exist in zookeeper");
+        throw new IllegalStateException(
+            "Instance id " + instanceID + " does not exist in zookeeper");
       }
-      throw new RuntimeException("Instance id " + instanceID + " pointed to by the name "
+      throw new IllegalStateException("Instance id " + instanceID + " pointed to by the name "
           + instanceName + " does not exist in zookeeper");
     }
     serverDirs = new ServerDirs(siteConfig, hadoopConf);
@@ -95,7 +98,7 @@
     try {
       volumeManager = VolumeManagerImpl.get(siteConfig, hadoopConf);
     } catch (IOException e) {
-      throw new IllegalStateException(e);
+      throw new UncheckedIOException(e);
     }
     serverDirs = new ServerDirs(siteConfig, hadoopConf);
     Path instanceIdPath = serverDirs.getInstanceIdLocation(volumeManager.getFirst());
@@ -114,7 +117,7 @@
     try {
       volumeManager = VolumeManagerImpl.get(siteConfig, hadoopConf);
     } catch (IOException e) {
-      throw new IllegalStateException(e);
+      throw new UncheckedIOException(e);
     }
     this.instanceID = instanceID;
     zooKeepers = config.get(Property.INSTANCE_ZK_HOST);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java b/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
deleted file mode 100644
index 3bde48f..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
+++ /dev/null
@@ -1,36 +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.server;
-
-import org.apache.accumulo.core.cli.ConfigOpts;
-
-import com.beust.jcommander.Parameter;
-
-public class ServerOpts extends ConfigOpts {
-
-  @Parameter(names = {"-a", "--address"}, description = "address to bind to")
-  private String address = null;
-
-  public String getAddress() {
-    if (address != null) {
-      return address;
-    }
-    return "0.0.0.0";
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
index 0f68a11..46c6fdb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.server;
 
-import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
@@ -62,14 +61,13 @@
   }
 
   @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 ConfigurationTypeHelper.getClassInstance(null, className, base);
   }
 
   @Override
   public <T> T instantiate(TableId tableId, String className, Class<T> base)
-      throws ReflectiveOperationException, IOException {
+      throws ReflectiveOperationException {
     String ctx = ClassLoaderUtil.tableContext(context.getTableConfiguration(tableId));
     return ConfigurationTypeHelper.getClassInstance(ctx, className, base);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
deleted file mode 100644
index 683ffaa..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ /dev/null
@@ -1,790 +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.server.client;
-
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-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.TabletLocator;
-import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
-import org.apache.accumulo.core.clientImpl.bulk.BulkImport;
-import org.apache.accumulo.core.clientImpl.thrift.ClientService;
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ByteSequence;
-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.dataImpl.thrift.TKeyExtent;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.spi.crypto.CryptoService;
-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.StopWatch;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.thrift.TServiceClient;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class BulkImporter {
-
-  private static final Logger log = LoggerFactory.getLogger(BulkImporter.class);
-
-  public static List<String> bulkLoad(ServerContext context, long tid, String tableId,
-      List<String> files, boolean setTime) throws IOException {
-    AssignmentStats stats = new BulkImporter(context, tid, tableId, setTime).importFiles(files);
-    List<String> result = new ArrayList<>();
-    for (Path p : stats.completeFailures.keySet()) {
-      result.add(p.toString());
-    }
-    return result;
-  }
-
-  private StopWatch<Timers> timer;
-
-  private enum Timers {
-    EXAMINE_MAP_FILES, QUERY_METADATA, IMPORT_MAP_FILES, SLEEP, TOTAL
-  }
-
-  private final ServerContext context;
-  private TableId tableId;
-  private long tid;
-  private boolean setTime;
-  private TableConfiguration tableConf;
-
-  public BulkImporter(ServerContext context, long tid, String tableId, boolean setTime) {
-    this.context = context;
-    this.tid = tid;
-    this.tableId = TableId.of(tableId);
-    this.setTime = setTime;
-    this.tableConf = context.getTableConfiguration(this.tableId);
-  }
-
-  public AssignmentStats importFiles(List<String> files) {
-
-    int numThreads = context.getConfiguration().getCount(Property.TSERV_BULK_PROCESS_THREADS);
-    int numAssignThreads =
-        context.getConfiguration().getCount(Property.TSERV_BULK_ASSIGNMENT_THREADS);
-
-    timer = new StopWatch<>(Timers.class);
-    timer.start(Timers.TOTAL);
-
-    final VolumeManager fs = context.getVolumeManager();
-
-    Set<Path> paths = new HashSet<>();
-    for (String file : files) {
-      paths.add(new Path(file));
-    }
-    AssignmentStats assignmentStats = new AssignmentStats(paths.size());
-
-    final Map<Path,List<KeyExtent>> completeFailures =
-        Collections.synchronizedSortedMap(new TreeMap<>());
-
-    ClientService.Client client = null;
-    final TabletLocator locator = TabletLocator.getLocator(context, tableId);
-
-    try {
-      final Map<Path,List<TabletLocation>> assignments =
-          Collections.synchronizedSortedMap(new TreeMap<>());
-
-      timer.start(Timers.EXAMINE_MAP_FILES);
-      ExecutorService threadPool = ThreadPools.getServerThreadPools()
-          .getPoolBuilder("findOverlapping").numCoreThreads(numThreads).build();
-
-      for (Path path : paths) {
-        final Path mapFile = path;
-        Runnable getAssignments = () -> {
-          List<TabletLocation> tabletsToAssignMapFileTo = Collections.emptyList();
-          try {
-            tabletsToAssignMapFileTo =
-                findOverlappingTablets(context, fs, locator, mapFile, tableConf.getCryptoService());
-          } catch (Exception ex) {
-            log.warn("Unable to find tablets that overlap file " + mapFile, ex);
-          }
-          log.debug("Map file {} found to overlap {} tablets", mapFile,
-              tabletsToAssignMapFileTo.size());
-          if (tabletsToAssignMapFileTo.isEmpty()) {
-            List<KeyExtent> empty = Collections.emptyList();
-            completeFailures.put(mapFile, empty);
-          } else {
-            assignments.put(mapFile, tabletsToAssignMapFileTo);
-          }
-
-        };
-        threadPool.execute(getAssignments);
-      }
-      threadPool.shutdown();
-      while (!threadPool.isTerminated()) {
-        try {
-          threadPool.awaitTermination(60, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-          throw new RuntimeException(e);
-        }
-      }
-      timer.stop(Timers.EXAMINE_MAP_FILES);
-
-      assignmentStats.attemptingAssignments(assignments);
-      Map<Path,List<KeyExtent>> assignmentFailures =
-          assignMapFiles(fs, assignments, paths, numAssignThreads, numThreads);
-      assignmentStats.assignmentsFailed(assignmentFailures);
-
-      Map<Path,Integer> failureCount = new TreeMap<>();
-
-      for (Entry<Path,List<KeyExtent>> entry : assignmentFailures.entrySet()) {
-        failureCount.put(entry.getKey(), 1);
-      }
-
-      long sleepTime = 2_000;
-      while (!assignmentFailures.isEmpty()) {
-        sleepTime = Math.min(sleepTime * 2, MINUTES.toMillis(1));
-        locator.invalidateCache();
-        // assumption about assignment failures is that it caused by a split
-        // happening or a missing location
-        //
-        // for splits we need to find children key extents that cover the
-        // same key range and are contiguous (no holes, no overlap)
-
-        timer.start(Timers.SLEEP);
-        sleepUninterruptibly(sleepTime, TimeUnit.MILLISECONDS);
-        timer.stop(Timers.SLEEP);
-
-        log.debug("Trying to assign {} map files that previously failed on some key extents",
-            assignmentFailures.size());
-        assignments.clear();
-
-        // for failed key extents, try to find children key extents to
-        // assign to
-        for (Entry<Path,List<KeyExtent>> entry : assignmentFailures.entrySet()) {
-          Iterator<KeyExtent> keListIter = entry.getValue().iterator();
-
-          List<TabletLocation> tabletsToAssignMapFileTo = new ArrayList<>();
-
-          while (keListIter.hasNext()) {
-            KeyExtent ke = keListIter.next();
-
-            timer.start(Timers.QUERY_METADATA);
-            try {
-              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(context, fs, locator,
-                  entry.getKey(), ke, tableConf.getCryptoService()));
-              keListIter.remove();
-            } catch (Exception ex) {
-              log.warn("Exception finding overlapping tablets, will retry tablet " + ke, ex);
-            }
-            timer.stop(Timers.QUERY_METADATA);
-          }
-
-          if (!tabletsToAssignMapFileTo.isEmpty()) {
-            assignments.put(entry.getKey(), tabletsToAssignMapFileTo);
-          }
-        }
-
-        assignmentStats.attemptingAssignments(assignments);
-        Map<Path,List<KeyExtent>> assignmentFailures2 =
-            assignMapFiles(fs, assignments, paths, numAssignThreads, numThreads);
-        assignmentStats.assignmentsFailed(assignmentFailures2);
-
-        // merge assignmentFailures2 into assignmentFailures
-        for (Entry<Path,List<KeyExtent>> entry : assignmentFailures2.entrySet()) {
-          assignmentFailures.get(entry.getKey()).addAll(entry.getValue());
-
-          Integer fc = failureCount.get(entry.getKey());
-          if (fc == null) {
-            fc = 0;
-          }
-
-          failureCount.put(entry.getKey(), fc + 1);
-        }
-
-        // remove map files that have no more key extents to assign
-        assignmentFailures.values().removeIf(List::isEmpty);
-
-        Set<Entry<Path,Integer>> failureIter = failureCount.entrySet();
-        for (Entry<Path,Integer> entry : failureIter) {
-          int retries = context.getConfiguration().getCount(Property.TSERV_BULK_RETRY);
-          if (entry.getValue() > retries && assignmentFailures.get(entry.getKey()) != null) {
-            log.error("Map file {} failed more than {} times, giving up.", entry.getKey(), retries);
-            completeFailures.put(entry.getKey(), assignmentFailures.get(entry.getKey()));
-            assignmentFailures.remove(entry.getKey());
-          }
-        }
-      }
-      assignmentStats.assignmentsAbandoned(completeFailures);
-      Set<Path> failedFailures = processFailures(completeFailures);
-      assignmentStats.unrecoveredMapFiles(failedFailures);
-
-      timer.stop(Timers.TOTAL);
-      printReport(paths);
-      return assignmentStats;
-    } finally {
-      if (client != null) {
-        ThriftUtil.close(client, context);
-      }
-    }
-  }
-
-  private void printReport(Set<Path> paths) {
-    long totalTime = 0;
-    for (Timers t : Timers.values()) {
-      if (t == Timers.TOTAL) {
-        continue;
-      }
-
-      totalTime += timer.get(t);
-    }
-    List<String> files = new ArrayList<>();
-    for (Path path : paths) {
-      files.add(path.getName());
-    }
-    Collections.sort(files);
-
-    log.debug("BULK IMPORT TIMING STATISTICS");
-    log.debug("Files: {}", files);
-    log.debug(String.format("Examine map files    : %,10.2f secs %6.2f%s",
-        timer.getSecs(Timers.EXAMINE_MAP_FILES),
-        100.0 * timer.get(Timers.EXAMINE_MAP_FILES) / timer.get(Timers.TOTAL), "%"));
-    log.debug(String.format("Query %-14s : %,10.2f secs %6.2f%s", MetadataTable.NAME,
-        timer.getSecs(Timers.QUERY_METADATA),
-        100.0 * timer.get(Timers.QUERY_METADATA) / timer.get(Timers.TOTAL), "%"));
-    log.debug(String.format("Import Map Files     : %,10.2f secs %6.2f%s",
-        timer.getSecs(Timers.IMPORT_MAP_FILES),
-        100.0 * timer.get(Timers.IMPORT_MAP_FILES) / timer.get(Timers.TOTAL), "%"));
-    log.debug(
-        String.format("Sleep                : %,10.2f secs %6.2f%s", timer.getSecs(Timers.SLEEP),
-            100.0 * timer.get(Timers.SLEEP) / timer.get(Timers.TOTAL), "%"));
-    log.debug(String.format("Misc                 : %,10.2f secs %6.2f%s",
-        (timer.get(Timers.TOTAL) - totalTime) / 1000.0,
-        100.0 * (timer.get(Timers.TOTAL) - totalTime) / timer.get(Timers.TOTAL), "%"));
-    log.debug(String.format("Total                : %,10.2f secs", timer.getSecs(Timers.TOTAL)));
-  }
-
-  private Set<Path> processFailures(Map<Path,List<KeyExtent>> completeFailures) {
-    // we should check if map file was not assigned to any tablets, then we
-    // should just move it; not currently being done?
-
-    Set<Entry<Path,List<KeyExtent>>> es = completeFailures.entrySet();
-
-    if (completeFailures.isEmpty()) {
-      return Collections.emptySet();
-    }
-
-    log.debug("The following map files failed ");
-
-    for (Entry<Path,List<KeyExtent>> entry : es) {
-      List<KeyExtent> extents = entry.getValue();
-
-      for (KeyExtent keyExtent : extents) {
-        log.debug("\t{} -> {}", entry.getKey(), keyExtent);
-      }
-    }
-
-    return Collections.emptySet();
-  }
-
-  private static class AssignmentInfo {
-    public AssignmentInfo(KeyExtent keyExtent, Long estSize) {
-      this.ke = keyExtent;
-      this.estSize = estSize;
-    }
-
-    KeyExtent ke;
-    long estSize;
-  }
-
-  private static List<KeyExtent> extentsOf(List<TabletLocation> locations) {
-    List<KeyExtent> result = new ArrayList<>(locations.size());
-    for (TabletLocation tl : locations) {
-      result.add(tl.tablet_extent);
-    }
-    return result;
-  }
-
-  private Map<Path,List<AssignmentInfo>> estimateSizes(final VolumeManager vm,
-      Map<Path,List<TabletLocation>> assignments, Collection<Path> paths, int numThreads) {
-
-    long t1 = System.currentTimeMillis();
-    final Map<Path,Long> mapFileSizes = new TreeMap<>();
-
-    try {
-      for (Path path : paths) {
-        FileSystem fs = vm.getFileSystemByPath(path);
-        mapFileSizes.put(path, fs.getContentSummary(path).getLength());
-      }
-    } catch (IOException e) {
-      log.error("Failed to get map files in for {}: {}", paths, e.getMessage(), e);
-      throw new RuntimeException(e);
-    }
-
-    final Map<Path,List<AssignmentInfo>> ais = Collections.synchronizedMap(new TreeMap<>());
-
-    ExecutorService threadPool = ThreadPools.getServerThreadPools().getPoolBuilder("estimateSizes")
-        .numCoreThreads(numThreads).build();
-
-    for (final Entry<Path,List<TabletLocation>> entry : assignments.entrySet()) {
-      if (entry.getValue().size() == 1) {
-        TabletLocation tabletLocation = entry.getValue().get(0);
-
-        // if the tablet completely contains the map file, there is no
-        // need to estimate its
-        // size
-        ais.put(entry.getKey(), Collections.singletonList(
-            new AssignmentInfo(tabletLocation.tablet_extent, mapFileSizes.get(entry.getKey()))));
-        continue;
-      }
-
-      Runnable estimationTask = () -> {
-        Map<KeyExtent,Long> estimatedSizes = null;
-
-        try {
-          Path mapFile = entry.getKey();
-          FileSystem ns = context.getVolumeManager().getFileSystemByPath(mapFile);
-
-          estimatedSizes = BulkImport.estimateSizes(context.getConfiguration(), mapFile,
-              mapFileSizes.get(entry.getKey()), extentsOf(entry.getValue()), ns, null,
-              tableConf.getCryptoService());
-        } catch (IOException e) {
-          log.warn("Failed to estimate map file sizes {}", e.getMessage());
-        }
-
-        if (estimatedSizes == null) {
-          // estimation failed, do a simple estimation
-          estimatedSizes = new TreeMap<>();
-          long estSize =
-              (long) (mapFileSizes.get(entry.getKey()) / (double) entry.getValue().size());
-          for (TabletLocation tl : entry.getValue()) {
-            estimatedSizes.put(tl.tablet_extent, estSize);
-          }
-        }
-
-        List<AssignmentInfo> assignmentInfoList = new ArrayList<>(estimatedSizes.size());
-
-        for (Entry<KeyExtent,Long> entry2 : estimatedSizes.entrySet()) {
-          assignmentInfoList.add(new AssignmentInfo(entry2.getKey(), entry2.getValue()));
-        }
-
-        ais.put(entry.getKey(), assignmentInfoList);
-      };
-
-      threadPool.execute(estimationTask);
-    }
-
-    threadPool.shutdown();
-
-    while (!threadPool.isTerminated()) {
-      try {
-        threadPool.awaitTermination(60, TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        log.error("Encountered InterruptedException while waiting for the threadPool to terminate.",
-            e);
-        throw new RuntimeException(e);
-      }
-    }
-
-    long t2 = System.currentTimeMillis();
-
-    log.debug(String.format("Estimated map files sizes in %6.2f secs", (t2 - t1) / 1000.0));
-
-    return ais;
-  }
-
-  private static Map<KeyExtent,String> locationsOf(Map<Path,List<TabletLocation>> assignments) {
-    Map<KeyExtent,String> result = new HashMap<>();
-    for (List<TabletLocation> entry : assignments.values()) {
-      for (TabletLocation tl : entry) {
-        result.put(tl.tablet_extent, tl.tablet_location);
-      }
-    }
-    return result;
-  }
-
-  private Map<Path,List<KeyExtent>> assignMapFiles(VolumeManager fs,
-      Map<Path,List<TabletLocation>> assignments, Collection<Path> paths, int numThreads,
-      int numMapThreads) {
-    timer.start(Timers.EXAMINE_MAP_FILES);
-    Map<Path,List<AssignmentInfo>> assignInfo =
-        estimateSizes(fs, assignments, paths, numMapThreads);
-    timer.stop(Timers.EXAMINE_MAP_FILES);
-
-    Map<Path,List<KeyExtent>> ret;
-
-    timer.start(Timers.IMPORT_MAP_FILES);
-    ret = assignMapFiles(assignInfo, locationsOf(assignments), numThreads);
-    timer.stop(Timers.IMPORT_MAP_FILES);
-
-    return ret;
-  }
-
-  private class AssignmentTask implements Runnable {
-    final Map<Path,List<KeyExtent>> assignmentFailures;
-    HostAndPort location;
-    private Map<KeyExtent,List<PathSize>> assignmentsPerTablet;
-
-    public AssignmentTask(Map<Path,List<KeyExtent>> assignmentFailures, String location,
-        Map<KeyExtent,List<PathSize>> assignmentsPerTablet) {
-      this.assignmentFailures = assignmentFailures;
-      this.location = HostAndPort.fromString(location);
-      this.assignmentsPerTablet = assignmentsPerTablet;
-    }
-
-    private void handleFailures(Collection<KeyExtent> failures, String message) {
-      failures.forEach(ke -> {
-        List<PathSize> mapFiles = assignmentsPerTablet.get(ke);
-        synchronized (assignmentFailures) {
-          mapFiles.forEach(pathSize -> assignmentFailures
-              .computeIfAbsent(pathSize.path, k -> new ArrayList<>()).add(ke));
-        }
-        log.info("Could not assign {} map files to tablet {} because : {}.  Will retry ...",
-            mapFiles.size(), ke, message);
-      });
-    }
-
-    @Override
-    public void run() {
-      HashSet<Path> uniqMapFiles = new HashSet<>();
-      for (List<PathSize> mapFiles : assignmentsPerTablet.values()) {
-        for (PathSize ps : mapFiles) {
-          uniqMapFiles.add(ps.path);
-        }
-      }
-
-      log.debug("Assigning {} map files to {} tablets at {}", uniqMapFiles.size(),
-          assignmentsPerTablet.size(), location);
-
-      try {
-        List<KeyExtent> failures = assignMapFiles(context, location, assignmentsPerTablet);
-        handleFailures(failures, "Not Serving Tablet");
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        handleFailures(assignmentsPerTablet.keySet(), e.getMessage());
-      }
-    }
-
-  }
-
-  private static class PathSize {
-    public PathSize(Path mapFile, long estSize) {
-      this.path = mapFile;
-      this.estSize = estSize;
-    }
-
-    Path path;
-    long estSize;
-
-    @Override
-    public String toString() {
-      return path + " " + estSize;
-    }
-  }
-
-  private Map<Path,List<KeyExtent>> assignMapFiles(Map<Path,List<AssignmentInfo>> assignments,
-      Map<KeyExtent,String> locations, int numThreads) {
-
-    // group assignments by tablet
-    Map<KeyExtent,List<PathSize>> assignmentsPerTablet = new TreeMap<>();
-    assignments.forEach((mapFile, tabletsToAssignMapFileTo) -> tabletsToAssignMapFileTo
-        .forEach(assignmentInfo -> assignmentsPerTablet
-            .computeIfAbsent(assignmentInfo.ke, k -> new ArrayList<>())
-            .add(new PathSize(mapFile, assignmentInfo.estSize))));
-
-    // group assignments by tabletserver
-
-    Map<Path,List<KeyExtent>> assignmentFailures = Collections.synchronizedMap(new TreeMap<>());
-
-    TreeMap<String,Map<KeyExtent,List<PathSize>>> assignmentsPerTabletServer = new TreeMap<>();
-
-    assignmentsPerTablet.forEach((ke, pathSizes) -> {
-      String location = locations.get(ke);
-      if (location == null) {
-        synchronized (assignmentFailures) {
-          pathSizes.forEach(pathSize -> assignmentFailures
-              .computeIfAbsent(pathSize.path, k -> new ArrayList<>()).add(ke));
-        }
-        log.warn(
-            "Could not assign {} map files to tablet {} because it had no location, will retry ...",
-            pathSizes.size(), ke);
-      } else {
-        assignmentsPerTabletServer.computeIfAbsent(location, k -> new TreeMap<>()).put(ke,
-            pathSizes);
-      }
-    });
-
-    ExecutorService threadPool = ThreadPools.getServerThreadPools().getPoolBuilder("submit")
-        .numCoreThreads(numThreads).build();
-
-    for (Entry<String,Map<KeyExtent,List<PathSize>>> entry : assignmentsPerTabletServer
-        .entrySet()) {
-      String location = entry.getKey();
-      threadPool.execute(new AssignmentTask(assignmentFailures, location, entry.getValue()));
-    }
-
-    threadPool.shutdown();
-
-    while (!threadPool.isTerminated()) {
-      try {
-        threadPool.awaitTermination(60, TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        log.error(
-            "Encountered InterruptedException while waiting for the thread pool to terminate.", e);
-        throw new RuntimeException(e);
-      }
-    }
-
-    return assignmentFailures;
-  }
-
-  private List<KeyExtent> assignMapFiles(ClientContext context, HostAndPort location,
-      Map<KeyExtent,List<PathSize>> assignmentsPerTablet)
-      throws AccumuloException, AccumuloSecurityException {
-    try {
-      long timeInMillis = context.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
-      TabletClientService.Iface client =
-          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeInMillis);
-      try {
-        HashMap<KeyExtent,Map<String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo>> files =
-            new HashMap<>();
-        for (Entry<KeyExtent,List<PathSize>> entry : assignmentsPerTablet.entrySet()) {
-          HashMap<String,org.apache.accumulo.core.dataImpl.thrift.MapFileInfo> tabletFiles =
-              new HashMap<>();
-          files.put(entry.getKey(), tabletFiles);
-
-          for (PathSize pathSize : entry.getValue()) {
-            org.apache.accumulo.core.dataImpl.thrift.MapFileInfo mfi =
-                new org.apache.accumulo.core.dataImpl.thrift.MapFileInfo(pathSize.estSize);
-            tabletFiles.put(pathSize.path.toString(), mfi);
-          }
-        }
-
-        log.debug("Asking {} to bulk load {}", location, files);
-        List<TKeyExtent> failures =
-            client.bulkImport(TraceUtil.traceInfo(), context.rpcCreds(), tid,
-                files.entrySet().stream()
-                    .collect(Collectors.toMap(entry -> entry.getKey().toThrift(), Entry::getValue)),
-                setTime);
-
-        return failures.stream().map(KeyExtent::fromThrift).collect(Collectors.toList());
-      } finally {
-        ThriftUtil.returnClient((TServiceClient) client, context);
-      }
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (Exception t) {
-      log.error("Encountered unknown exception in assignMapFiles.", t);
-      throw new AccumuloException(t);
-    }
-  }
-
-  public static List<TabletLocation> findOverlappingTablets(ServerContext context, VolumeManager fs,
-      TabletLocator locator, Path file, CryptoService cs) throws Exception {
-    return findOverlappingTablets(context, fs, locator, file, null, null, cs);
-  }
-
-  public static List<TabletLocation> findOverlappingTablets(ServerContext context, VolumeManager fs,
-      TabletLocator locator, Path file, KeyExtent failed, CryptoService cs) throws Exception {
-    locator.invalidateCache(failed);
-    Text start = getStartRowForExtent(failed);
-    return findOverlappingTablets(context, fs, locator, file, start, failed.endRow(), cs);
-  }
-
-  protected static Text getStartRowForExtent(KeyExtent extent) {
-    Text start = extent.prevEndRow();
-    if (start != null) {
-      start = new Text(start);
-      // ACCUMULO-3967 We want the first possible key in this tablet, not the following row from the
-      // previous tablet
-      start.append(byte0, 0, 1);
-    }
-    return start;
-  }
-
-  static final byte[] byte0 = {0};
-
-  public static List<TabletLocation> findOverlappingTablets(ServerContext context, VolumeManager vm,
-      TabletLocator locator, Path file, Text startRow, Text endRow, CryptoService cs)
-      throws Exception {
-    List<TabletLocation> result = new ArrayList<>();
-    Collection<ByteSequence> columnFamilies = Collections.emptyList();
-    String filename = file.toString();
-    // log.debug(filename + " finding overlapping tablets " + startRow + " -> " + endRow);
-    FileSystem fs = vm.getFileSystemByPath(file);
-    try (FileSKVIterator reader =
-        FileOperations.getInstance().newReaderBuilder().forFile(filename, fs, fs.getConf(), cs)
-            .withTableConfiguration(context.getConfiguration()).seekToBeginning().build()) {
-      Text row = startRow;
-      if (row == null) {
-        row = new Text();
-      }
-      while (true) {
-        // log.debug(filename + " Seeking to row " + row);
-        reader.seek(new Range(row, null), columnFamilies, false);
-        if (!reader.hasTop()) {
-          // log.debug(filename + " not found");
-          break;
-        }
-        row = reader.getTopKey().getRow();
-        TabletLocation tabletLocation = locator.locateTablet(context, row, false, true);
-        // log.debug(filename + " found row " + row + " at location " + tabletLocation);
-        result.add(tabletLocation);
-        row = tabletLocation.tablet_extent.endRow();
-        if (row != null && (endRow == null || row.compareTo(endRow) < 0)) {
-          row = new Text(row);
-          row.append(byte0, 0, byte0.length);
-        } else {
-          break;
-        }
-      }
-    }
-    // log.debug(filename + " to be sent to " + result);
-    return result;
-  }
-
-  public static class AssignmentStats {
-    private Map<KeyExtent,Integer> counts;
-    private int numUniqueMapFiles;
-    private Map<Path,List<KeyExtent>> completeFailures = null;
-    private Set<Path> failedFailures = null;
-
-    AssignmentStats(int fileCount) {
-      counts = new HashMap<>();
-      numUniqueMapFiles = fileCount;
-    }
-
-    void attemptingAssignments(Map<Path,List<TabletLocation>> assignments) {
-      for (Entry<Path,List<TabletLocation>> entry : assignments.entrySet()) {
-        for (TabletLocation tl : entry.getValue()) {
-
-          Integer count = getCount(tl.tablet_extent);
-
-          counts.put(tl.tablet_extent, count + 1);
-        }
-      }
-    }
-
-    void assignmentsFailed(Map<Path,List<KeyExtent>> assignmentFailures) {
-      for (Entry<Path,List<KeyExtent>> entry : assignmentFailures.entrySet()) {
-        for (KeyExtent ke : entry.getValue()) {
-
-          Integer count = getCount(ke);
-
-          counts.put(ke, count - 1);
-        }
-      }
-    }
-
-    void assignmentsAbandoned(Map<Path,List<KeyExtent>> completeFailures) {
-      this.completeFailures = completeFailures;
-    }
-
-    private Integer getCount(KeyExtent parent) {
-      Integer count = counts.get(parent);
-
-      if (count == null) {
-        count = 0;
-      }
-      return count;
-    }
-
-    void unrecoveredMapFiles(Set<Path> failedFailures) {
-      this.failedFailures = failedFailures;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder();
-      int totalAssignments = 0;
-      int tabletsImportedTo = 0;
-
-      int min = Integer.MAX_VALUE, max = Integer.MIN_VALUE;
-
-      for (Entry<KeyExtent,Integer> entry : counts.entrySet()) {
-        totalAssignments += entry.getValue();
-        if (entry.getValue() > 0) {
-          tabletsImportedTo++;
-        }
-
-        if (entry.getValue() < min) {
-          min = entry.getValue();
-        }
-
-        if (entry.getValue() > max) {
-          max = entry.getValue();
-        }
-      }
-
-      double stddev = 0;
-
-      for (Entry<KeyExtent,Integer> entry : counts.entrySet()) {
-        stddev += Math.pow(entry.getValue() - totalAssignments / (double) counts.size(), 2);
-      }
-
-      stddev = stddev / counts.size();
-      stddev = Math.sqrt(stddev);
-
-      Set<KeyExtent> failedTablets = new HashSet<>();
-      for (List<KeyExtent> ft : completeFailures.values()) {
-        failedTablets.addAll(ft);
-      }
-
-      sb.append("BULK IMPORT ASSIGNMENT STATISTICS\n");
-      sb.append(String.format("# of map files            : %,10d%n", numUniqueMapFiles));
-      sb.append(String.format("# map files with failures : %,10d %6.2f%s%n",
-          completeFailures.size(), completeFailures.size() * 100.0 / numUniqueMapFiles, "%"));
-      sb.append(String.format("# failed failed map files : %,10d %s%n", failedFailures.size(),
-          failedFailures.isEmpty() ? "" : " <-- THIS IS BAD"));
-      sb.append(String.format("# of tablets              : %,10d%n", counts.size()));
-      sb.append(String.format("# tablets imported to     : %,10d %6.2f%s%n", tabletsImportedTo,
-          tabletsImportedTo * 100.0 / counts.size(), "%"));
-      sb.append(String.format("# tablets with failures   : %,10d %6.2f%s%n", failedTablets.size(),
-          failedTablets.size() * 100.0 / counts.size(), "%"));
-      sb.append(String.format("min map files per tablet  : %,10d%n", min));
-      sb.append(String.format("max map files per tablet  : %,10d%n", max));
-      sb.append(String.format("avg map files per tablet  : %,10.2f (std dev = %.2f)%n",
-          totalAssignments / (double) counts.size(), stddev));
-      return sb.toString();
-    }
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 76f0692..dd7fbe7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -31,7 +31,6 @@
 import java.util.SortedSet;
 
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -44,6 +43,7 @@
 import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.clientImpl.thrift.TDiskUsage;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
@@ -53,14 +53,12 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.store.NamespacePropKey;
 import org.apache.accumulo.server.conf.store.SystemPropKey;
@@ -357,7 +355,7 @@
       case DEFAULT:
         return conf(credentials, context.getDefaultConfiguration());
     }
-    throw new RuntimeException("Unexpected configuration type " + type);
+    throw new IllegalArgumentException("Unexpected configuration type " + type);
   }
 
   @Override
@@ -405,31 +403,6 @@
   }
 
   @Override
-  public List<String> bulkImportFiles(TInfo tinfo, final TCredentials credentials, final long tid,
-      final String tableId, final List<String> files, final String errorDir, final boolean setTime)
-      throws ThriftSecurityException, ThriftTableOperationException, TException {
-    try {
-      if (!security.canPerformSystemActions(credentials)) {
-        throw new AccumuloSecurityException(credentials.getPrincipal(),
-            SecurityErrorCode.PERMISSION_DENIED);
-      }
-      bulkImportStatus.updateBulkImportStatus(files, BulkImportState.INITIAL);
-      log.debug("Got request to bulk import files to table({}): {}", tableId, files);
-
-      bulkImportStatus.updateBulkImportStatus(files, BulkImportState.PROCESSING);
-      try {
-        return BulkImporter.bulkLoad(context, tid, tableId, files, setTime);
-      } finally {
-        bulkImportStatus.removeBulkImportStatus(files);
-      }
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    } catch (Exception ex) {
-      throw new TException(ex);
-    }
-  }
-
-  @Override
   public boolean isActive(TInfo tinfo, long tid) {
     return transactionWatcher.isActive(tid);
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionInfo.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionInfo.java
index 5006539..57d4d24 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionInfo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionInfo.java
@@ -38,12 +38,14 @@
   private final String localityGroup;
   private final long entriesRead;
   private final long entriesWritten;
+  private final long timesPaused;
   private final TCompactionReason reason;
 
   CompactionInfo(FileCompactor compactor) {
     this.localityGroup = compactor.getCurrentLocalityGroup();
     this.entriesRead = compactor.getEntriesRead();
     this.entriesWritten = compactor.getEntriesWritten();
+    this.timesPaused = compactor.getTimesPaused();
     this.reason = compactor.getReason();
     this.compactor = compactor;
   }
@@ -64,11 +66,15 @@
     return entriesWritten;
   }
 
+  public long getTimesPaused() {
+    return timesPaused;
+  }
+
   public Thread getThread() {
     return compactor.thread;
   }
 
-  public String getOutputFile() {
+  public StoredTabletFile getOutputFile() {
     return compactor.getOutputFile();
   }
 
@@ -96,10 +102,11 @@
           iterSetting.getName()));
       iterOptions.put(iterSetting.getName(), iterSetting.getOptions());
     }
-    List<String> files = compactor.getFilesToCompact().stream().map(StoredTabletFile::getPathStr)
-        .collect(Collectors.toList());
+    List<String> files = compactor.getFilesToCompact().stream()
+        .map(StoredTabletFile::getNormalizedPathStr).collect(Collectors.toList());
     return new ActiveCompaction(compactor.extent.toThrift(),
-        System.currentTimeMillis() - compactor.getStartTime(), files, compactor.getOutputFile(),
-        type, reason, localityGroup, entriesRead, entriesWritten, iiList, iterOptions);
+        System.currentTimeMillis() - compactor.getStartTime(), files,
+        compactor.getOutputFile().getMetadataPath(), type, reason, localityGroup, entriesRead,
+        entriesWritten, iiList, iterOptions, timesPaused);
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionStats.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionStats.java
index e3ccfdd..9704726 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionStats.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionStats.java
@@ -22,10 +22,12 @@
   private long entriesRead;
   private long entriesWritten;
   private long fileSize;
+  private int timesPaused;
 
-  public CompactionStats(long er, long ew) {
+  public CompactionStats(long er, long ew, int tp) {
     this.setEntriesRead(er);
     this.setEntriesWritten(ew);
+    this.setTimesPaused(tp);
   }
 
   public CompactionStats() {}
@@ -46,9 +48,18 @@
     return entriesWritten;
   }
 
+  public long getTimesPaused() {
+    return timesPaused;
+  }
+
+  public void setTimesPaused(int timesPaused) {
+    this.timesPaused = timesPaused;
+  }
+
   public void add(CompactionStats mcs) {
     this.entriesRead += mcs.entriesRead;
     this.entriesWritten += mcs.entriesWritten;
+    this.timesPaused += mcs.timesPaused;
   }
 
   public void setFileSize(long fileSize) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
index 47e75fd..ff90bd5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/CompactionWatcher.java
@@ -62,6 +62,12 @@
     this.config = config;
   }
 
+  @SuppressWarnings("deprecation")
+  private static long getCompactionWarnTime(AccumuloConfiguration config) {
+    return config.getTimeInMillis(
+        config.resolve(Property.COMPACTION_WARN_TIME, Property.TSERV_COMPACTION_WARN_TIME));
+  }
+
   @Override
   public void run() {
     List<CompactionInfo> runningCompactions = FileCompactor.getRunningCompactions();
@@ -98,7 +104,7 @@
     // remove any compaction that completed or made progress
     observedCompactions.keySet().retainAll(newKeys);
 
-    long warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+    long warnTime = getCompactionWarnTime(config);
 
     // check for stuck compactions
     for (ObservedCompactionInfo oci : observedCompactions.values()) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
index 7c16351..5fb112d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
@@ -31,6 +31,7 @@
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -51,12 +52,11 @@
 import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
 import org.apache.accumulo.core.iteratorsImpl.system.ColumnFamilySkippingIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator;
+import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
-import org.apache.accumulo.core.iteratorsImpl.system.TimeSettingIterator;
-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.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason;
@@ -67,6 +67,7 @@
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.iterators.SystemIteratorEnvironment;
+import org.apache.accumulo.server.mem.LowMemoryDetector.DetectionScope;
 import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReportingIterator;
 import org.apache.accumulo.server.problems.ProblemReports;
@@ -107,7 +108,7 @@
   }
 
   private final Map<StoredTabletFile,DataFileValue> filesToCompact;
-  private final TabletFile outputFile;
+  private final ReferencedTabletFile outputFile;
   private final boolean propagateDeletes;
   private final AccumuloConfiguration acuTableConf;
   private final CompactionEnv env;
@@ -115,6 +116,7 @@
   protected final KeyExtent extent;
   private final List<IteratorSetting> iterators;
   private final CryptoService cryptoService;
+  private final PausedCompactionMetrics metrics;
 
   // things to report
   private String currentLocalityGroup = "";
@@ -122,6 +124,7 @@
 
   private final AtomicLong entriesRead = new AtomicLong(0);
   private final AtomicLong entriesWritten = new AtomicLong(0);
+  private final AtomicInteger timesPaused = new AtomicInteger(0);
   private final DateFormat dateFormatter = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
 
   // a unique id to identify a compactor
@@ -144,6 +147,7 @@
   private void clearStats() {
     entriesRead.set(0);
     entriesWritten.set(0);
+    timesPaused.set(0);
   }
 
   protected static final Set<FileCompactor> runningCompactions =
@@ -162,9 +166,9 @@
   }
 
   public FileCompactor(ServerContext context, KeyExtent extent,
-      Map<StoredTabletFile,DataFileValue> files, TabletFile outputFile, boolean propagateDeletes,
-      CompactionEnv env, List<IteratorSetting> iterators, AccumuloConfiguration tableConfiguation,
-      CryptoService cs) {
+      Map<StoredTabletFile,DataFileValue> files, ReferencedTabletFile outputFile,
+      boolean propagateDeletes, CompactionEnv env, List<IteratorSetting> iterators,
+      AccumuloConfiguration tableConfiguation, CryptoService cs, PausedCompactionMetrics metrics) {
     this.context = context;
     this.extent = extent;
     this.fs = context.getVolumeManager();
@@ -175,6 +179,7 @@
     this.env = env;
     this.iterators = iterators;
     this.cryptoService = cs;
+    this.metrics = metrics;
 
     startTime = System.currentTimeMillis();
   }
@@ -187,8 +192,8 @@
     return extent;
   }
 
-  protected String getOutputFile() {
-    return outputFile.toString();
+  protected StoredTabletFile getOutputFile() {
+    return outputFile.insert();
   }
 
   protected Map<String,Set<ByteSequence>> getLocalityGroups(AccumuloConfiguration acuTableConf)
@@ -224,14 +229,15 @@
 
       final boolean isMinC = env.getIteratorScope() == IteratorUtil.IteratorScope.minc;
 
-      final boolean dropCacheBehindOutput = !RootTable.ID.equals(this.extent.tableId())
-          && !MetadataTable.ID.equals(this.extent.tableId())
-          && ((isMinC && acuTableConf.getBoolean(Property.TABLE_MINC_OUTPUT_DROP_CACHE))
-              || (!isMinC && acuTableConf.getBoolean(Property.TABLE_MAJC_OUTPUT_DROP_CACHE)));
+      final boolean dropCacheBehindOutput =
+          !AccumuloTable.ROOT.tableId().equals(this.extent.tableId())
+              && !AccumuloTable.METADATA.tableId().equals(this.extent.tableId())
+              && ((isMinC && acuTableConf.getBoolean(Property.TABLE_MINC_OUTPUT_DROP_CACHE))
+                  || (!isMinC && acuTableConf.getBoolean(Property.TABLE_MAJC_OUTPUT_DROP_CACHE)));
 
-      WriterBuilder outBuilder = fileFactory.newWriterBuilder()
-          .forFile(outputFile.getMetaInsert(), ns, ns.getConf(), cryptoService)
-          .withTableConfiguration(acuTableConf).withRateLimiter(env.getWriteLimiter());
+      WriterBuilder outBuilder =
+          fileFactory.newWriterBuilder().forFile(outputFile, ns, ns.getConf(), cryptoService)
+              .withTableConfiguration(acuTableConf).withRateLimiter(env.getWriteLimiter());
       if (dropCacheBehindOutput) {
         outBuilder.dropCachesBehind();
       }
@@ -272,10 +278,11 @@
 
       log.trace(String.format(
           "Compaction %s %,d read | %,d written | %,6d entries/sec"
-              + " | %,6.3f secs | %,12d bytes | %9.3f byte/sec",
+              + " | %,6.3f secs | %,12d bytes | %9.3f byte/sec | %,d paused",
           extent, majCStats.getEntriesRead(), majCStats.getEntriesWritten(),
           (int) (majCStats.getEntriesRead() / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0,
-          mfwTmp.getLength(), mfwTmp.getLength() / ((t2 - t1) / 1000.0)));
+          mfwTmp.getLength(), mfwTmp.getLength() / ((t2 - t1) / 1000.0),
+          majCStats.getTimesPaused()));
 
       majCStats.setFileSize(mfwTmp.getLength());
       return majCStats;
@@ -333,41 +340,38 @@
 
     List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(filesToCompact.size());
 
-    for (TabletFile mapFile : filesToCompact.keySet()) {
+    for (StoredTabletFile dataFile : filesToCompact.keySet()) {
       try {
 
         FileOperations fileFactory = FileOperations.getInstance();
-        FileSystem fs = this.fs.getFileSystemByPath(mapFile.getPath());
+        FileSystem fs = this.fs.getFileSystemByPath(dataFile.getPath());
         FileSKVIterator reader;
 
-        reader = fileFactory.newReaderBuilder()
-            .forFile(mapFile.getPathStr(), fs, fs.getConf(), cryptoService)
+        reader = fileFactory.newReaderBuilder().forFile(dataFile, fs, fs.getConf(), cryptoService)
             .withTableConfiguration(acuTableConf).withRateLimiter(env.getReadLimiter())
             .dropCachesBehind().build();
 
         readers.add(reader);
 
-        SortedKeyValueIterator<Key,Value> iter = new ProblemReportingIterator(context,
-            extent.tableId(), mapFile.getPathStr(), false, reader);
+        InterruptibleIterator iter = new ProblemReportingIterator(context, extent.tableId(),
+            dataFile.getNormalizedPathStr(), false, reader);
 
-        if (filesToCompact.get(mapFile).isTimeSet()) {
-          iter = new TimeSettingIterator(iter, filesToCompact.get(mapFile).getTime());
-        }
+        iter = filesToCompact.get(dataFile).wrapFileIterator(iter);
 
         iters.add(iter);
 
       } catch (Exception e) {
 
-        ProblemReports.getInstance(context).report(
-            new ProblemReport(extent.tableId(), ProblemType.FILE_READ, mapFile.getPathStr(), e));
+        ProblemReports.getInstance(context).report(new ProblemReport(extent.tableId(),
+            ProblemType.FILE_READ, dataFile.getNormalizedPathStr(), e));
 
-        log.warn("Some problem opening map file {} {}", mapFile, e.getMessage(), e);
-        // failed to open some map file... close the ones that were opened
+        log.warn("Some problem opening data file {} {}", dataFile, e.getMessage(), e);
+        // failed to open some data file... close the ones that were opened
         for (FileSKVIterator reader : readers) {
           try {
             reader.close();
           } catch (Exception e2) {
-            log.warn("Failed to close map file", e2);
+            log.warn("Failed to close data file", e2);
           }
         }
 
@@ -376,7 +380,7 @@
         if (e instanceof IOException) {
           throw (IOException) e;
         }
-        throw new IOException("Failed to open map data files", e);
+        throw new IOException("Failed to open data files", e);
       }
     }
 
@@ -416,9 +420,31 @@
         mfw.startDefaultLocalityGroup();
       }
 
+      DetectionScope scope =
+          env.getIteratorScope() == IteratorScope.minc ? DetectionScope.MINC : DetectionScope.MAJC;
       Span writeSpan = TraceUtil.startSpan(this.getClass(), "write");
       try (Scope write = writeSpan.makeCurrent()) {
         while (itr.hasTop() && env.isCompactionEnabled()) {
+
+          while (context.getLowMemoryDetector().isRunningLowOnMemory(context, scope, () -> {
+            return !extent.isMeta();
+          }, () -> {
+            log.info("Pausing compaction because low on memory, extent: {}", extent);
+            timesPaused.incrementAndGet();
+            if (scope == DetectionScope.MINC) {
+              metrics.incrementMinCPause();
+            } else {
+              metrics.incrementMajCPause();
+            }
+            try {
+              Thread.sleep(500);
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+              throw new IllegalStateException(
+                  "Interrupted while waiting for low memory condition to resolve", e);
+            }
+          })) {}
+
           mfw.append(itr.getTopKey(), itr.getTopValue());
           itr.next();
           entriesCompacted++;
@@ -446,12 +472,12 @@
         }
 
       } finally {
-        CompactionStats lgMajcStats = new CompactionStats(citr.getCount(), entriesCompacted);
+        CompactionStats lgMajcStats =
+            new CompactionStats(citr.getCount(), entriesCompacted, timesPaused.get());
         majCStats.add(lgMajcStats);
         writeSpan.end();
       }
-
-    } catch (Exception e) {
+    } catch (IOException | CompactionCanceledException e) {
       TraceUtil.setException(compactSpan, e, true);
       throw e;
     } finally {
@@ -460,7 +486,7 @@
         try {
           reader.close();
         } catch (Exception e) {
-          log.warn("Failed to close map file", e);
+          log.warn("Failed to close data file", e);
         }
       }
       compactSpan.end();
@@ -487,6 +513,10 @@
     return entriesWritten.get();
   }
 
+  long getTimesPaused() {
+    return timesPaused.get();
+  }
+
   long getStartTime() {
     return startTime;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/PausedCompactionMetrics.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/PausedCompactionMetrics.java
new file mode 100644
index 0000000..3d1a07b
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/PausedCompactionMetrics.java
@@ -0,0 +1,47 @@
+/*
+ * 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.server.compaction;
+
+import org.apache.accumulo.core.metrics.MetricsProducer;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.MeterRegistry;
+
+public class PausedCompactionMetrics implements MetricsProducer {
+
+  private Counter majcPauses;
+  private Counter mincPauses;
+
+  public void incrementMinCPause() {
+    mincPauses.increment();
+  }
+
+  public void incrementMajCPause() {
+    majcPauses.increment();
+  }
+
+  @Override
+  public void registerMetrics(MeterRegistry registry) {
+    majcPauses = Counter.builder(METRICS_MAJC_PAUSED).description("major compaction pause count")
+        .register(registry);
+    mincPauses = Counter.builder(METRICS_MINC_PAUSED).description("minor compactor pause count")
+        .register(registry);
+  }
+
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
index 020e436..2d5d83a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
@@ -18,9 +18,9 @@
  */
 package org.apache.accumulo.server.compaction;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.time.Duration.ofMillis;
 
-import java.util.concurrent.TimeUnit;
+import java.time.Duration;
 
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.Retry.NeedsRetryDelay;
@@ -78,8 +78,8 @@
     } else {
       builder = Retry.builder().maxRetries(maxNumRetries);
     }
-    this.retry = builder.retryAfter(start, MILLISECONDS).incrementBy(0, MILLISECONDS)
-        .maxWait(maxWaitTime, MILLISECONDS).backOffFactor(2).logInterval(1, TimeUnit.MINUTES)
+    this.retry = builder.retryAfter(Duration.ofMillis(start)).incrementBy(Duration.ZERO)
+        .maxWait(Duration.ofMillis(maxWaitTime)).backOffFactor(2).logInterval(Duration.ofMinutes(1))
         .createRetry();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/CheckCompactionConfig.java b/server/base/src/main/java/org/apache/accumulo/server/conf/CheckCompactionConfig.java
index 6926fd5..65ffd01 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/CheckCompactionConfig.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/CheckCompactionConfig.java
@@ -94,7 +94,7 @@
     }
 
     AccumuloConfiguration config = SiteConfiguration.fromFile(path.toFile()).build();
-    var servicesConfig = new CompactionServicesConfig(config, log::warn);
+    var servicesConfig = new CompactionServicesConfig(config);
     ServiceEnvironment senv = createServiceEnvironment(config);
 
     Set<String> defaultServices = Set.of(DEFAULT, META, ROOT);
@@ -114,7 +114,8 @@
       CompactionPlanner planner = plannerClass.getDeclaredConstructor().newInstance();
 
       var initParams = new CompactionPlannerInitParams(CompactionServiceId.of(serviceId),
-          servicesConfig.getOptions().get(serviceId), senv);
+          servicesConfig.getPlannerPrefix(serviceId), servicesConfig.getOptions().get(serviceId),
+          senv);
 
       planner.init(initParams);
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
index d59abbb..72ddd1d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java
@@ -116,6 +116,7 @@
         var conf =
             new TableConfiguration(context, tableId, getNamespaceConfigurationForTable(tableId));
         ConfigCheckUtil.validate(conf, "table id: " + tableId.toString());
+
         return conf;
       }
       return null;
@@ -127,7 +128,7 @@
     try {
       namespaceId = context.getNamespaceId(tableId);
     } catch (TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
     return tableParentConfigs.get(tableId, key -> getNamespaceConfiguration(namespaceId));
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
index d25cbd2..9f733e8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/store/impl/ZooPropStore.java
@@ -37,8 +37,6 @@
 import org.apache.accumulo.server.conf.store.PropChangeListener;
 import org.apache.accumulo.server.conf.store.PropStore;
 import org.apache.accumulo.server.conf.store.PropStoreKey;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.util.ConfigTransformer;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.checkerframework.checker.nullness.qual.NonNull;
@@ -167,11 +165,6 @@
       return props;
     }
 
-    if (propStoreKey instanceof SystemPropKey) {
-      return new ConfigTransformer(zrw, codec, propStoreWatcher).transform(propStoreKey,
-          propStoreKey.getPath(), false);
-    }
-
     throw new IllegalStateException(
         "Invalid request for " + propStoreKey + ", the property node does not exist");
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java
deleted file mode 100644
index b6400e5..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgrader.java
+++ /dev/null
@@ -1,144 +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.server.conf.util;
-
-import static org.apache.accumulo.core.Constants.ZCONFIG;
-
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.NamespaceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.cli.ServerUtilOpts;
-import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
-import org.apache.accumulo.server.conf.store.NamespacePropKey;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.store.TablePropKey;
-import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
-import org.apache.accumulo.server.conf.store.impl.ReadyMonitor;
-import org.apache.accumulo.start.spi.KeywordExecutable;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.auto.service.AutoService;
-import com.google.common.annotations.VisibleForTesting;
-
-@AutoService(KeywordExecutable.class)
-public class ConfigPropertyUpgrader implements KeywordExecutable {
-
-  private static final Logger log = LoggerFactory.getLogger(ConfigPropertyUpgrader.class);
-
-  private final static VersionedPropCodec codec = VersionedPropCodec.getDefault();
-
-  public ConfigPropertyUpgrader() {}
-
-  public static void main(String[] args) throws Exception {
-    new ConfigPropertyUpgrader().execute(args);
-  }
-
-  @Override
-  public String keyword() {
-    return "config-upgrade";
-  }
-
-  @Override
-  public String description() {
-    return "converts properties store in ZooKeeper to 2.1 format";
-  }
-
-  @Override
-  public void execute(final String[] args) throws Exception {
-    ServerUtilOpts opts = new ServerUtilOpts();
-    opts.parseArgs(ConfigPropertyUpgrader.class.getName(), args);
-
-    ServerContext context = opts.getServerContext();
-
-    doUpgrade(context.getInstanceID(), context.getZooReaderWriter());
-  }
-
-  public void doUpgrade(final InstanceId instanceId, final ZooReaderWriter zrw) {
-
-    ReadyMonitor readyMonitor = new ReadyMonitor(ConfigPropertyUpgrader.class.getSimpleName(),
-        zrw.getSessionTimeout() * 2L);
-    PropStoreWatcher nullWatcher = new PropStoreWatcher(readyMonitor);
-
-    ConfigTransformer transformer = new ConfigTransformer(zrw, codec, nullWatcher);
-
-    upgradeSysProps(instanceId, transformer);
-    upgradeNamespaceProps(instanceId, zrw, transformer);
-    upgradeTableProps(instanceId, zrw, transformer);
-  }
-
-  @VisibleForTesting
-  void upgradeSysProps(final InstanceId instanceId, final ConfigTransformer transformer) {
-    log.info("Upgrade system config properties for {}", instanceId);
-    String legacyPath = ZooUtil.getRoot(instanceId) + ZCONFIG;
-    transformer.transform(SystemPropKey.of(instanceId), legacyPath, false);
-  }
-
-  @VisibleForTesting
-  void upgradeNamespaceProps(final InstanceId instanceId, final ZooReaderWriter zrw,
-      final ConfigTransformer transformer) {
-    String zkPathNamespaceBase = ZooUtil.getRoot(instanceId) + Constants.ZNAMESPACES;
-    try {
-      // sort is cosmetic - only improves readability and consistency in logs
-      Set<String> namespaces = new TreeSet<>(zrw.getChildren(zkPathNamespaceBase));
-      for (String namespace : namespaces) {
-        String legacyPath = zkPathNamespaceBase + "/" + namespace + Constants.ZCONF_LEGACY;
-        log.info("Upgrading namespace {} base path: {}", namespace, legacyPath);
-        transformer.transform(NamespacePropKey.of(instanceId, NamespaceId.of(namespace)),
-            legacyPath, true);
-      }
-    } catch (KeeperException ex) {
-      throw new IllegalStateException(
-          "Failed to read namespaces from ZooKeeper for path: " + zkPathNamespaceBase, ex);
-    } catch (InterruptedException ex) {
-      throw new IllegalStateException(
-          "Interrupted reading namespaces from ZooKeeper for path: " + zkPathNamespaceBase, ex);
-    }
-  }
-
-  @VisibleForTesting
-  void upgradeTableProps(final InstanceId instanceId, final ZooReaderWriter zrw,
-      ConfigTransformer transformer) {
-    String zkPathTableBase = ZooUtil.getRoot(instanceId) + Constants.ZTABLES;
-    try {
-      // sort is cosmetic - only improves readability and consistency in logs
-      Set<String> tables = new TreeSet<>(zrw.getChildren(zkPathTableBase));
-      for (String table : tables) {
-        String legacyPath = zkPathTableBase + "/" + table + Constants.ZCONF_LEGACY;
-        log.info("Upgrading table {} base path: {}", table, legacyPath);
-        transformer.transform(TablePropKey.of(instanceId, TableId.of(table)), legacyPath, true);
-      }
-    } catch (KeeperException ex) {
-      throw new IllegalStateException(
-          "Failed to read tables from ZooKeeper for path: " + zkPathTableBase, ex);
-    } catch (InterruptedException ex) {
-      throw new IllegalStateException(
-          "Interrupted reading tables from ZooKeeper for path: " + zkPathTableBase, ex);
-    }
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
deleted file mode 100644
index 53195c2..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ConfigTransformer.java
+++ /dev/null
@@ -1,443 +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.server.conf.util;
-
-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 java.io.IOException;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.util.DurationFormat;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.Retry;
-import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
-import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.PropStoreKey;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
-import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.checkerframework.checker.nullness.qual.NonNull;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Read legacy properties (pre 2.1) from ZooKeeper and transform them into the single node format.
- * The encoded properties are stored in ZooKeeper and then the legacy property ZooKeeper nodes are
- * deleted.
- */
-public class ConfigTransformer {
-
-  private static final Logger log = LoggerFactory.getLogger(ConfigTransformer.class);
-
-  private final ZooReaderWriter zrw;
-  private final VersionedPropCodec codec;
-  private final PropStoreWatcher propStoreWatcher;
-  private final Retry retry;
-
-  /**
-   * Instantiate a transformer instance.
-   *
-   * @param zrw a ZooReaderWriter
-   * @param codec the codec used to encode to the single-node format.
-   * @param propStoreWatcher the watcher registered to receive future notifications of changes to
-   *        the encoded property node.
-   */
-  public ConfigTransformer(final ZooReaderWriter zrw, VersionedPropCodec codec,
-      final PropStoreWatcher propStoreWatcher) {
-    this.zrw = zrw;
-    this.codec = codec;
-    this.propStoreWatcher = propStoreWatcher;
-
-    // default - allow for a conservative max delay of about a minute
-    retry =
-        Retry.builder().maxRetries(15).retryAfter(250, MILLISECONDS).incrementBy(500, MILLISECONDS)
-            .maxWait(5, SECONDS).backOffFactor(1.75).logInterval(3, MINUTES).createRetry();
-
-  }
-
-  public ConfigTransformer(final ZooReaderWriter zrw, VersionedPropCodec codec,
-      final PropStoreWatcher propStoreWatcher, final Retry retry) {
-    this.zrw = zrw;
-    this.codec = codec;
-    this.propStoreWatcher = propStoreWatcher;
-    this.retry = retry;
-  }
-
-  /**
-   * Transform the properties for the provided prop cache key.
-   *
-   * @return the encoded properties.
-   */
-  public VersionedProperties transform(final PropStoreKey<?> propStoreKey, final String legacyPath,
-      final boolean deleteLegacyNode) {
-    VersionedProperties exists = checkNeedsTransform(propStoreKey);
-    if (exists != null) {
-      return exists;
-    }
-    TransformToken token = TransformToken.createToken(legacyPath, zrw);
-    return transform(propStoreKey, token, legacyPath, deleteLegacyNode);
-  }
-
-  // Allow external (mocked) TransformToken to be used
-  @VisibleForTesting
-  VersionedProperties transform(final PropStoreKey<?> propStoreKey, final TransformToken token,
-      final String legacyPath, final boolean deleteLegacyNode) {
-    log.trace("checking for legacy property upgrade transform for {}", propStoreKey);
-    VersionedProperties results;
-    Instant start = Instant.now();
-    try {
-
-      // check for node - just return if it exists.
-      results = checkNeedsTransform(propStoreKey);
-      if (results != null) {
-        return results;
-      }
-
-      while (!token.haveTokenOwnership()) {
-        try {
-          retry.useRetry();
-          retry.waitForNextAttempt(log, "transform property at " + propStoreKey.getPath());
-          // look and return node if created while trying to token.
-          log.trace("own the token - look for existing encoded node at: {}",
-              propStoreKey.getPath());
-          results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
-          if (results != null) {
-            log.trace(
-                "Found existing node with properties after getting token at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
-                propStoreKey, results.getDataVersion(), results.getTimestamp());
-            return results;
-          }
-          // still does not exist - try again.
-          token.getTokenOwnership();
-        } catch (InterruptedException ex) {
-          Thread.currentThread().interrupt();
-          throw new IllegalStateException("Failed to hold transform token for " + propStoreKey, ex);
-        } catch (IllegalStateException ex) {
-          throw new IllegalStateException("Failed to hold transform token for " + propStoreKey, ex);
-        }
-      }
-
-      Set<LegacyPropNode> upgradeNodes = readLegacyProps(legacyPath);
-      if (upgradeNodes.size() == 0) {
-        log.trace("No existing legacy props {}, skipping conversion, writing default prop node",
-            propStoreKey);
-        return writeNode(propStoreKey, Map.of());
-      }
-
-      upgradeNodes = convertDeprecatedProps(propStoreKey, upgradeNodes);
-
-      results = writeConverted(propStoreKey, upgradeNodes);
-
-      if (results == null) {
-        throw new IllegalStateException("Could not create properties for " + propStoreKey);
-      }
-
-      // validate token still valid before deletion.
-      if (!token.validateToken()) {
-        throw new IllegalStateException(
-            "legacy conversion failed. Lost transform token for " + propStoreKey);
-      }
-
-      Pair<Integer,Integer> deleteCounts = deleteLegacyProps(upgradeNodes);
-      log.info("property transform for {} took {} ms, delete count: {}, error count: {}",
-          propStoreKey, new DurationFormat(Duration.between(start, Instant.now()).toMillis(), ""),
-          deleteCounts.getFirst(), deleteCounts.getSecond());
-
-      return results;
-
-    } catch (Exception ex) {
-      log.info("Issue on upgrading legacy properties for: " + propStoreKey, ex);
-    } finally {
-      token.releaseToken();
-      if (deleteLegacyNode) {
-        log.trace("Delete legacy property base node: {}", legacyPath);
-        try {
-          zrw.delete(legacyPath);
-        } catch (KeeperException.NotEmptyException ex) {
-          log.info("Delete for legacy prop node {} - not empty", legacyPath);
-        } catch (KeeperException | InterruptedException ex) {
-          Thread.currentThread().interrupt();
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * If the config node exists, return the properties, otherwise return null. ZooKeeper exceptions
-   * are ignored. Interrupt exceptions will be propagated as IllegalStateExceptions.
-   *
-   * @param propStoreKey the prop key for that identifies the configuration node.
-   * @return the existing encoded properties if present, null if they do not.
-   */
-  private VersionedProperties checkNeedsTransform(PropStoreKey<?> propStoreKey) {
-    try { // check for node - just return if it exists.
-      VersionedProperties results = ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
-      if (results != null) {
-        log.trace(
-            "Found existing node with properties at {}. skipping legacy prop conversion - version: {}, timestamp: {}",
-            propStoreKey, results.getDataVersion(), results.getTimestamp());
-        return results;
-      }
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted during zookeeper read", ex);
-    } catch (IOException | KeeperException ex) {
-      log.trace("node for {} not found for upgrade", propStoreKey);
-    }
-    return null;
-  }
-
-  private Set<LegacyPropNode> convertDeprecatedProps(PropStoreKey<?> propStoreKey,
-      Set<LegacyPropNode> upgradeNodes) {
-
-    if (!(propStoreKey instanceof SystemPropKey)) {
-      return upgradeNodes;
-    }
-
-    Set<LegacyPropNode> renamedNodes = new TreeSet<>();
-
-    for (LegacyPropNode original : upgradeNodes) {
-      var finalName = DeprecatedPropertyUtil.getReplacementName(original.getPropName(),
-          (log, replacement) -> log
-              .info("Automatically renaming deprecated property '{}' with its replacement '{}'"
-                  + " in ZooKeeper configuration upgrade.", original, replacement));
-      LegacyPropNode renamed = new LegacyPropNode(original.getPath(), finalName, original.getData(),
-          original.getNodeVersion());
-      renamedNodes.add(renamed);
-    }
-    return renamedNodes;
-  }
-
-  private @NonNull Set<LegacyPropNode> readLegacyProps(final String basePath) {
-
-    Set<LegacyPropNode> legacyProps = new TreeSet<>();
-
-    // strip leading slash
-    var tokenName = TransformToken.TRANSFORM_TOKEN.substring(1);
-
-    try {
-      List<String> childNames = zrw.getChildren(basePath);
-      for (String propName : childNames) {
-        log.trace("processing ZooKeeper child node: {} at path: {}", propName, basePath);
-        if (tokenName.equals(propName)) {
-          continue;
-        }
-
-        log.trace("Adding: {} to list for legacy conversion", propName);
-
-        var path = basePath + "/" + propName;
-        Stat stat = new Stat();
-        byte[] bytes = zrw.getData(path, stat);
-
-        try {
-          LegacyPropNode node;
-          if (stat.getDataLength() > 0) {
-            node = new LegacyPropNode(path, propName, new String(bytes, UTF_8), stat.getVersion());
-          } else {
-            node = new LegacyPropNode(path, propName, "", stat.getVersion());
-          }
-          legacyProps.add(node);
-        } catch (IllegalStateException ex) {
-          log.warn("Skipping invalid property at path " + path, ex);
-        }
-      }
-
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("Failed to read legacy props due to ZooKeeper error", ex);
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException(
-          "Failed to read legacy props due to interrupt read from ZooKeeper", ex);
-    }
-    return legacyProps;
-  }
-
-  private Pair<Integer,Integer> deleteLegacyProps(Set<LegacyPropNode> nodes) {
-    int deleteCount = 0;
-    int errorCount = 0;
-    for (LegacyPropNode n : nodes) {
-      try {
-        log.trace("Delete legacy prop at path: {}, data version: {}", n.getPath(),
-            n.getNodeVersion());
-        deleteCount++;
-        zrw.deleteStrict(n.getPath(), n.getNodeVersion());
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-        throw new IllegalStateException("interrupt received during upgrade node clean-up", ex);
-      } catch (KeeperException ex) {
-        errorCount++;
-        log.info("Failed to delete node during upgrade clean-up", ex);
-      }
-    }
-    return new Pair<>(deleteCount, errorCount);
-  }
-
-  private @Nullable VersionedProperties writeConverted(final PropStoreKey<?> propStoreKey,
-      final Set<LegacyPropNode> nodes) {
-    final Map<String,String> props = new HashMap<>();
-    nodes.forEach(node -> props.put(node.getPropName(), node.getData()));
-
-    VersionedProperties vProps;
-    try {
-      vProps = writeNode(propStoreKey, props);
-    } catch (InterruptedException | KeeperException ex) {
-      if (ex instanceof InterruptedException) {
-        Thread.currentThread().interrupt();
-      }
-      throw new IllegalStateException(
-          "failed to create node for " + propStoreKey + " on conversion", ex);
-    }
-
-    if (!validateWrite(propStoreKey, vProps)) {
-      log.trace("Failed property conversion validation for: {}", propStoreKey);
-      // failed validation
-      return null;
-    }
-
-    return vProps;
-  }
-
-  private VersionedProperties writeNode(final PropStoreKey<?> propStoreKey,
-      final Map<String,String> props) throws InterruptedException, KeeperException {
-    VersionedProperties vProps;
-    try {
-      String path = propStoreKey.getPath();
-      log.trace("Writing converted properties to ZooKeeper path: {} for key: {}", path,
-          propStoreKey);
-      Stat currStat = zrw.getStatus(path);
-      if (currStat == null || currStat.getDataLength() == 0) {
-        // no node or node with no props stored
-        vProps = new VersionedProperties(props);
-        zrw.putPrivatePersistentData(path, codec.toBytes(vProps),
-            ZooUtil.NodeExistsPolicy.OVERWRITE);
-      }
-      return ZooPropStore.readFromZk(propStoreKey, propStoreWatcher, zrw);
-    } catch (IOException ex) {
-      throw new IllegalStateException(
-          "failed to create node for " + propStoreKey + " on conversion", ex);
-    }
-  }
-
-  private boolean validateWrite(final PropStoreKey<?> propStoreKey,
-      final VersionedProperties vProps) {
-    try {
-      Stat stat = zrw.getStatus(propStoreKey.getPath(), propStoreWatcher);
-      if (stat == null) {
-        throw new IllegalStateException(
-            "failed to get stat to validate created node for " + propStoreKey);
-      }
-      log.debug("Property conversion validation - version received: {}, version expected: {}",
-          stat.getVersion(), vProps.getDataVersion());
-      return stat.getVersion() == vProps.getDataVersion();
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("failed to validate created node for " + propStoreKey, ex);
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("failed to validate created node for " + propStoreKey, ex);
-    }
-  }
-
-  /**
-   * Immutable container for legacy ZooKeeper property node information.
-   */
-  private static class LegacyPropNode implements Comparable<LegacyPropNode> {
-    private final String path;
-    private final String propName;
-    private final String data;
-    private final int nodeVersion;
-
-    /**
-     * An immutable instance of legacy ZooKeeper property node information. It holds the property
-     * and the node stat for later comparison to enable detection of ZooKeeper node changes. If the
-     * legacy property name has been deprecated, the property is renamed and the conversion is noted
-     * in the log.
-     *
-     * @param path the ZooKeeper path
-     * @param propName the property name - if deprecated it will be stored as the updated name and
-     *        the conversion logged.
-     * @param data the property value
-     * @param nodeVersion the ZooKeeper stat data version.
-     */
-    public LegacyPropNode(@NonNull final String path, final String propName, final String data,
-        final int nodeVersion) {
-      this.path = path;
-      this.propName = propName;
-      this.data = data;
-      this.nodeVersion = nodeVersion;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public String getPropName() {
-      return propName;
-    }
-
-    public String getData() {
-      return data;
-    }
-
-    public int getNodeVersion() {
-      return nodeVersion;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-      LegacyPropNode that = (LegacyPropNode) o;
-      return path.equals(that.path);
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(path);
-    }
-
-    @Override
-    public int compareTo(LegacyPropNode other) {
-      return path.compareTo(other.path);
-    }
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
deleted file mode 100644
index 9241913..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/TransformToken.java
+++ /dev/null
@@ -1,207 +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.server.conf.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.util.Arrays;
-import java.util.Objects;
-import java.util.UUID;
-
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.checkerframework.checker.nullness.qual.NonNull;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Provides a token used in property conversion. The token is used to limit the number of processes
- * that try to create a property node and transform the legacy property format to the 2.1 encoded
- * properties. Processes do not queue for a token (using a sequential node) - processes should look
- * for the token to exist, and if present wait and then periodically re-check for the property node
- * to be created by the process that created / has the token.
- * <p>
- * Features
- * <ul>
- * <li>Uses ephemeral node that will be removed if transform process terminates without
- * completing</li>
- * <li>Watcher not necessary - the existence of the node and uuid in data sufficient to detect
- * changes.</li>
- * </ul>
- */
-public class TransformToken {
-  public static final String TRANSFORM_TOKEN = "/transform_token";
-  private static final Logger log = LoggerFactory.getLogger(TransformToken.class);
-  private final TokenUUID tokenUUID = new TokenUUID();
-  private final String path;
-  private final ZooReaderWriter zrw;
-  private boolean haveToken = false;
-
-  private TransformToken(final @NonNull String basePath, final ZooReaderWriter zrw) {
-    path = basePath + TRANSFORM_TOKEN;
-    this.zrw = zrw;
-
-    boolean t = getTokenOwnership();
-    log.trace("created token - token held: {}", t);
-  }
-
-  /**
-   * Create a lock node in ZooKeeper using an ephemeral node. Will not throw and exception except on
-   * an interrupt. If the lock node is created, the returned lock will be locked. If another lock
-   * already exists, the lock is unlocked and the caller can decide to either wait for the resource
-   * to be created by the thread that created the lock, or try calling to {@code lock} to succeed
-   *
-   * @param path the parent node of the legacy properties and the associated property children
-   *        nodes.
-   * @param zrw a ZooReaderWriter
-   * @return an TransformLock instance.
-   * @throws IllegalStateException is the lock creation fails due to an underlying ZooKeeper
-   *         exception.
-   */
-  public static TransformToken createToken(final @NonNull String path, final ZooReaderWriter zrw) {
-    return new TransformToken(path, zrw);
-  }
-
-  /**
-   * Create and try to establish ownership (hold the token). Token ownership can be tested with
-   * {@link #haveTokenOwnership() haveTokenOwnership}
-   *
-   * @return true if able to get ownership, false otherwise.
-   */
-  public boolean getTokenOwnership() {
-    if (haveToken) {
-      return true;
-    }
-    try {
-      // existence check should be lighter-weight than failing on NODE_EXISTS exception
-      if (zrw.exists(path)) {
-        return false;
-      }
-      // if this completes this thread has created the lock
-      zrw.putEphemeralData(path, tokenUUID.asBytes());
-      log.trace("wrote property transform token: {} - {}", path, tokenUUID);
-      haveToken = true;
-      return true;
-    } catch (KeeperException ex) {
-      log.debug(
-          "Failed to write transform token for " + path + " another process may have created one",
-          ex);
-      return false;
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted getting transform token", ex);
-    }
-  }
-
-  /**
-   * Return the token ownership status
-   *
-   * @return true if this instance has ownership of the token, false otherwise.
-   */
-  public boolean haveTokenOwnership() {
-    return haveToken;
-  }
-
-  /**
-   * Verify ownership is still valid while holding the token.
-   *
-   * @return true if token is still owned, false otherwise
-   */
-  public boolean validateToken() {
-    try {
-      byte[] readId = zrw.getData(path);
-      log.trace("validate token: read: {} - expected: {}", readId, tokenUUID);
-      return Arrays.equals(readId, tokenUUID.asBytes());
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("Failed to validate token", ex);
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted while validating token", ex);
-    }
-  }
-
-  /**
-   * If the token was created by this instance, the uuid of this instance and the uuid stored in the
-   * ZooKeeper data will match.
-   */
-  public void releaseToken() {
-    try {
-      if (log.isTraceEnabled()) {
-        log.trace("releaseToken called - {} - exists in ZooKeeper: {}", path, zrw.exists(path));
-      }
-
-      Stat stat = new Stat();
-      byte[] readId = zrw.getData(path, stat);
-      if (!Arrays.equals(readId, tokenUUID.asBytes())) {
-        throw new IllegalStateException(
-            "tried to release a token that was not held by current thread");
-      }
-
-      if (log.isTraceEnabled()) {
-        log.trace("releaseToken read id: {} - exists: {}", readId, zrw.exists(path));
-      }
-
-      // make sure we are deleting the same node version just checked.
-      zrw.deleteStrict(path, stat.getVersion());
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("Failed to release transform lock for " + path, ex);
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted getting transform token", ex);
-    }
-    haveToken = false;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    TransformToken that = (TransformToken) o;
-    return path.equals(that.path) && Arrays.equals(tokenUUID.asBytes(), that.tokenUUID.asBytes());
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(path, Arrays.hashCode(tokenUUID.asBytes()));
-  }
-
-  @Override
-  public String toString() {
-    return "TransformLock{ path='" + path + "', locked='" + haveToken + "' id=" + tokenUUID + "'}'";
-  }
-
-  private static class TokenUUID {
-    private final String id = UUID.randomUUID().toString();
-    private final byte[] idBytes = id.getBytes(UTF_8);
-
-    public byte[] asBytes() {
-      return idBytes;
-    }
-
-    @Override
-    public String toString() {
-      return "TransformToken{id='" + id + '}';
-    }
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index ced97d0..0f4a361 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -31,10 +31,11 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.constraints.Constraint;
 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.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
@@ -45,6 +46,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.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
@@ -83,6 +85,9 @@
           ServerColumnFamily.FLUSH_COLUMN,
           ServerColumnFamily.COMPACT_COLUMN);
 
+  @SuppressWarnings("deprecation")
+  private static final Text CHOPPED = ChoppedColumnFamily.NAME;
+
   private static final Set<Text> validColumnFams =
       Set.of(BulkFileColumnFamily.NAME,
           LogColumnFamily.NAME,
@@ -91,9 +96,11 @@
           CurrentLocationColumnFamily.NAME,
           LastLocationColumnFamily.NAME,
           FutureLocationColumnFamily.NAME,
-          ChoppedColumnFamily.NAME,
           ClonedColumnFamily.NAME,
-          ExternalCompactionColumnFamily.NAME);
+          ExternalCompactionColumnFamily.NAME,
+          CHOPPED,
+          MergedColumnFamily.NAME
+      );
   // @formatter:on
 
   private static boolean isValidColumn(ColumnUpdate cu) {
@@ -124,6 +131,19 @@
     return lst;
   }
 
+  /*
+   * Validates the data file metadata is valid for a StoredTabletFile.
+   */
+  private static ArrayList<Short> validateDataFileMetadata(ArrayList<Short> violations,
+      String metadata) {
+    try {
+      StoredTabletFile.validate(metadata);
+    } catch (RuntimeException e) {
+      violations = addViolation(violations, 9);
+    }
+    return violations;
+  }
+
   @Override
   public List<Short> check(Environment env, Mutation mutation) {
     final ServerContext context = ((SystemEnvironment) env).getServerContext();
@@ -177,7 +197,7 @@
     }
 
     // ensure row is not less than Constants.METADATA_TABLE_ID
-    if (new Text(row).compareTo(new Text(MetadataTable.ID.canonical())) < 0) {
+    if (new Text(row).compareTo(new Text(AccumuloTable.METADATA.tableId().canonical())) < 0) {
       violations = addViolation(violations, 5);
     }
 
@@ -193,11 +213,15 @@
         continue;
       }
 
-      if (columnUpdate.getValue().length == 0 && !columnFamily.equals(ScanFileColumnFamily.NAME)) {
+      if (columnUpdate.getValue().length == 0 && !(columnFamily.equals(ScanFileColumnFamily.NAME)
+          || columnFamily.equals(LogColumnFamily.NAME))) {
         violations = addViolation(violations, 6);
       }
 
       if (columnFamily.equals(DataFileColumnFamily.NAME)) {
+        violations = validateDataFileMetadata(violations,
+            new String(columnUpdate.getColumnQualifier(), UTF_8));
+
         try {
           DataFileValue dfv = new DataFileValue(columnUpdate.getValue());
 
@@ -208,9 +232,22 @@
           violations = addViolation(violations, 1);
         }
       } else if (columnFamily.equals(ScanFileColumnFamily.NAME)) {
-        // Do nothing if ScanFile ref
+        violations = validateDataFileMetadata(violations,
+            new String(columnUpdate.getColumnQualifier(), UTF_8));
       } else if (columnFamily.equals(BulkFileColumnFamily.NAME)) {
         if (!columnUpdate.isDeleted() && !checkedBulk) {
+          /*
+           * This needs to be re-worked after Issue https://github.com/apache/accumulo/issues/3505
+           * is done.
+           *
+           * That issue will reorganizes this class and make things more efficient so we are not
+           * looping over the same mutation more than once like in this case. The below check is
+           * commented out for now because the violation check is already done when creating
+           * StoredTabletFiles so it isn't needed here anymore violations =
+           * validateDataFileMetadata(violations, new String(columnUpdate.getColumnQualifier(),
+           * UTF_8));
+           */
+
           // splits, which also write the time reference, are allowed to write this reference even
           // when
           // the transaction is not running because the other half of the tablet is holding a
@@ -224,8 +261,8 @@
           // See ACCUMULO-1230.
           boolean isLocationMutation = false;
 
-          HashSet<Text> dataFiles = new HashSet<>();
-          HashSet<Text> loadedFiles = new HashSet<>();
+          HashSet<StoredTabletFile> dataFiles = new HashSet<>();
+          HashSet<StoredTabletFile> loadedFiles = new HashSet<>();
 
           String tidString = new String(columnUpdate.getValue(), UTF_8);
           int otherTidCount = 0;
@@ -237,9 +274,20 @@
                 .equals(CurrentLocationColumnFamily.NAME)) {
               isLocationMutation = true;
             } else if (new Text(update.getColumnFamily()).equals(DataFileColumnFamily.NAME)) {
-              dataFiles.add(new Text(update.getColumnQualifier()));
+              try {
+                // This actually validates for a second time as the loop already validates
+                // if a DataFileColumnFamily, this will likely be fixed as part of
+                // https://github.com/apache/accumulo/issues/3505
+                dataFiles.add(StoredTabletFile.of(new Text(update.getColumnQualifier())));
+              } catch (RuntimeException e) {
+                violations = addViolation(violations, 9);
+              }
             } else if (new Text(update.getColumnFamily()).equals(BulkFileColumnFamily.NAME)) {
-              loadedFiles.add(new Text(update.getColumnQualifier()));
+              try {
+                loadedFiles.add(StoredTabletFile.of(new Text(update.getColumnQualifier())));
+              } catch (RuntimeException e) {
+                violations = addViolation(violations, 9);
+              }
 
               if (!new String(update.getValue(), UTF_8).equals(tidString)) {
                 otherTidCount++;
@@ -321,13 +369,16 @@
       case 4:
         return "Invalid metadata row format";
       case 5:
-        return "Row can not be less than " + MetadataTable.ID;
+        return "Row can not be less than " + AccumuloTable.METADATA.tableId();
       case 6:
-        return "Empty values are not allowed for any " + MetadataTable.NAME + " column";
+        return "Empty values are not allowed for any " + AccumuloTable.METADATA.tableName()
+            + " column";
       case 7:
         return "Lock not held in zookeeper by writer";
       case 8:
         return "Bulk load mutation contains either inconsistent files or multiple fateTX ids";
+      case 9:
+        return "Invalid data file metadata format";
     }
     return null;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java b/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java
index 79f379a..5dc8868 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java
@@ -46,8 +46,7 @@
 import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator.DataSource;
-import org.apache.accumulo.core.iteratorsImpl.system.TimeSettingIterator;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.util.threads.ThreadPools;
@@ -57,11 +56,10 @@
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
 
 public class FileManager {
 
@@ -72,10 +70,10 @@
   private static class OpenReader implements Comparable<OpenReader> {
     long releaseTime;
     FileSKVIterator reader;
-    String fileName;
+    StoredTabletFile file;
 
-    public OpenReader(String fileName, FileSKVIterator reader) {
-      this.fileName = fileName;
+    public OpenReader(StoredTabletFile file, FileSKVIterator reader) {
+      this.file = file;
       this.reader = reader;
       this.releaseTime = System.currentTimeMillis();
     }
@@ -95,12 +93,12 @@
 
     @Override
     public int hashCode() {
-      return fileName.hashCode();
+      return file.hashCode();
     }
   }
 
-  private Map<String,List<OpenReader>> openFiles;
-  private HashMap<FileSKVIterator,String> reservedReaders;
+  private Map<StoredTabletFile,List<OpenReader>> openFiles;
+  private HashMap<FileSKVIterator,StoredTabletFile> reservedReaders;
 
   private Semaphore filePermits;
 
@@ -123,9 +121,9 @@
       // determine which files to close in a sync block, and then close the
       // files outside of the sync block
       synchronized (FileManager.this) {
-        Iterator<Entry<String,List<OpenReader>>> iter = openFiles.entrySet().iterator();
+        Iterator<Entry<StoredTabletFile,List<OpenReader>>> iter = openFiles.entrySet().iterator();
         while (iter.hasNext()) {
-          Entry<String,List<OpenReader>> entry = iter.next();
+          Entry<StoredTabletFile,List<OpenReader>> entry = iter.next();
           List<OpenReader> ofl = entry.getValue();
 
           for (Iterator<OpenReader> oflIter = ofl.iterator(); oflIter.hasNext();) {
@@ -174,7 +172,7 @@
         this.context.getConfiguration().getTimeInMillis(Property.TSERV_SLOW_FILEPERMIT_MILLIS);
   }
 
-  private static int countReaders(Map<String,List<OpenReader>> files) {
+  private static int countReaders(Map<StoredTabletFile,List<OpenReader>> files) {
     int count = 0;
 
     for (List<OpenReader> list : files.values()) {
@@ -188,7 +186,7 @@
 
     ArrayList<OpenReader> openReaders = new ArrayList<>();
 
-    for (Entry<String,List<OpenReader>> entry : openFiles.entrySet()) {
+    for (Entry<StoredTabletFile,List<OpenReader>> entry : openFiles.entrySet()) {
       openReaders.addAll(entry.getValue());
     }
 
@@ -199,13 +197,13 @@
     for (int i = 0; i < numToTake && i < openReaders.size(); i++) {
       OpenReader or = openReaders.get(i);
 
-      List<OpenReader> ofl = openFiles.get(or.fileName);
+      List<OpenReader> ofl = openFiles.get(or.file);
       if (!ofl.remove(or)) {
-        throw new RuntimeException("Failed to remove open reader that should have been there");
+        throw new IllegalStateException("Failed to remove open reader that should have been there");
       }
 
       if (ofl.isEmpty()) {
-        openFiles.remove(or.fileName);
+        openFiles.remove(or.file);
       }
 
       ret.add(or.reader);
@@ -224,10 +222,10 @@
     }
   }
 
-  private List<String> takeOpenFiles(Collection<String> files,
-      Map<FileSKVIterator,String> readersReserved) {
-    List<String> filesToOpen = Collections.emptyList();
-    for (String file : files) {
+  private List<StoredTabletFile> takeOpenFiles(Collection<StoredTabletFile> files,
+      Map<FileSKVIterator,StoredTabletFile> readersReserved) {
+    List<StoredTabletFile> filesToOpen = Collections.emptyList();
+    for (StoredTabletFile file : files) {
       List<OpenReader> ofl = openFiles.get(file);
       if (ofl != null && !ofl.isEmpty()) {
         OpenReader openReader = ofl.remove(ofl.size() - 1);
@@ -245,8 +243,9 @@
     return filesToOpen;
   }
 
-  private Map<FileSKVIterator,String> reserveReaders(KeyExtent tablet, Collection<String> files,
-      boolean continueOnFailure, CacheProvider cacheProvider) throws IOException {
+  private Map<FileSKVIterator,StoredTabletFile> reserveReaders(KeyExtent tablet,
+      Collection<StoredTabletFile> files, boolean continueOnFailure, CacheProvider cacheProvider)
+      throws IOException {
 
     if (!tablet.isMeta() && files.size() >= maxOpen) {
       throw new IllegalArgumentException("requested files exceeds max open");
@@ -256,9 +255,9 @@
       return Collections.emptyMap();
     }
 
-    List<String> filesToOpen = null;
+    List<StoredTabletFile> filesToOpen = null;
     List<FileSKVIterator> filesToClose = Collections.emptyList();
-    Map<FileSKVIterator,String> readersReserved = new HashMap<>();
+    Map<FileSKVIterator,StoredTabletFile> readersReserved = new HashMap<>();
 
     if (!tablet.isMeta()) {
       long start = System.currentTimeMillis();
@@ -298,24 +297,20 @@
     closeReaders(filesToClose);
 
     // open any files that need to be opened
-    for (String file : filesToOpen) {
+    for (StoredTabletFile file : filesToOpen) {
       try {
-        if (!file.contains(":")) {
-          throw new IllegalArgumentException("Expected uri, got : " + file);
-        }
-        Path path = new Path(file);
-        FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
+        FileSystem ns = context.getVolumeManager().getFileSystemByPath(file.getPath());
         // log.debug("Opening "+file + " path " + path);
         var tableConf = context.getTableConfiguration(tablet.tableId());
         FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf(), tableConf.getCryptoService())
+            .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
             .withTableConfiguration(tableConf).withCacheProvider(cacheProvider)
             .withFileLenCache(fileLenCache).build();
         readersReserved.put(reader, file);
       } catch (Exception e) {
 
         ProblemReports.getInstance(context)
-            .report(new ProblemReport(tablet.tableId(), ProblemType.FILE_READ, file, e));
+            .report(new ProblemReport(tablet.tableId(), ProblemType.FILE_READ, file.toString(), e));
 
         if (continueOnFailure) {
           // release the permit for the file that failed to open
@@ -367,10 +362,9 @@
       }
 
       for (FileSKVIterator reader : readers) {
-        String fileName = reservedReaders.remove(reader);
+        StoredTabletFile file = reservedReaders.remove(reader);
         if (!sawIOException) {
-          openFiles.computeIfAbsent(fileName, k -> new ArrayList<>())
-              .add(new OpenReader(fileName, reader));
+          openFiles.computeIfAbsent(file, k -> new ArrayList<>()).add(new OpenReader(file, reader));
         }
       }
     }
@@ -392,10 +386,10 @@
     private ArrayList<FileDataSource> deepCopies;
     private boolean current = true;
     private IteratorEnvironment env;
-    private String file;
+    private StoredTabletFile file;
     private AtomicBoolean iflag;
 
-    FileDataSource(String file, SortedKeyValueIterator<Key,Value> iter) {
+    FileDataSource(StoredTabletFile file, SortedKeyValueIterator<Key,Value> iter) {
       this.file = file;
       this.iter = iter;
       this.deepCopies = new ArrayList<>();
@@ -482,7 +476,7 @@
       }
     }
 
-    private Map<FileSKVIterator,String> openFiles(List<String> files)
+    private Map<FileSKVIterator,StoredTabletFile> openFiles(List<StoredTabletFile> files)
         throws TooManyFilesException, IOException {
       // one tablet can not open more than maxOpen files, otherwise it could get stuck
       // forever waiting on itself to release files
@@ -494,26 +488,28 @@
                 + maxOpen + " tablet = " + tablet);
       }
 
-      Map<FileSKVIterator,String> newlyReservedReaders =
+      Map<FileSKVIterator,StoredTabletFile> newlyReservedReaders =
           reserveReaders(tablet, files, continueOnFailure, cacheProvider);
 
       tabletReservedReaders.addAll(newlyReservedReaders.keySet());
       return newlyReservedReaders;
     }
 
-    public synchronized List<InterruptibleIterator> openFiles(Map<TabletFile,DataFileValue> files,
-        boolean detachable, SamplerConfigurationImpl samplerConfig) throws IOException {
+    public synchronized List<InterruptibleIterator> openFiles(
+        Map<StoredTabletFile,DataFileValue> files, boolean detachable,
+        SamplerConfigurationImpl samplerConfig) throws IOException {
 
-      Map<FileSKVIterator,String> newlyReservedReaders = openFiles(
-          files.keySet().stream().map(TabletFile::getPathStr).collect(Collectors.toList()));
+      Map<FileSKVIterator,StoredTabletFile> newlyReservedReaders =
+          openFiles(new ArrayList<>(files.keySet()));
 
       ArrayList<InterruptibleIterator> iters = new ArrayList<>();
 
-      boolean sawTimeSet = files.values().stream().anyMatch(DataFileValue::isTimeSet);
+      boolean someIteratorsWillWrap =
+          files.values().stream().anyMatch(DataFileValue::willWrapIterator);
 
-      for (Entry<FileSKVIterator,String> entry : newlyReservedReaders.entrySet()) {
+      for (Entry<FileSKVIterator,StoredTabletFile> entry : newlyReservedReaders.entrySet()) {
         FileSKVIterator source = entry.getKey();
-        String filename = entry.getValue();
+        StoredTabletFile file = entry.getValue();
         InterruptibleIterator iter;
 
         if (samplerConfig != null) {
@@ -523,15 +519,13 @@
           }
         }
 
-        iter = new ProblemReportingIterator(context, tablet.tableId(), filename, continueOnFailure,
-            detachable ? getSsi(filename, source) : source);
+        iter = new ProblemReportingIterator(context, tablet.tableId(), file.toString(),
+            continueOnFailure, detachable ? getSsi(file, source) : source);
 
-        if (sawTimeSet) {
+        if (someIteratorsWillWrap) {
           // constructing FileRef is expensive so avoid if not needed
-          DataFileValue value = files.get(new TabletFile(new Path(filename)));
-          if (value.isTimeSet()) {
-            iter = new TimeSettingIterator(iter, value.getTime());
-          }
+          DataFileValue value = files.get(file);
+          iter = value.wrapFileIterator(iter);
         }
 
         iters.add(iter);
@@ -540,8 +534,8 @@
       return iters;
     }
 
-    private SourceSwitchingIterator getSsi(String filename, FileSKVIterator source) {
-      FileDataSource fds = new FileDataSource(filename, source);
+    private SourceSwitchingIterator getSsi(StoredTabletFile file, FileSKVIterator source) {
+      FileDataSource fds = new FileDataSource(file, source);
       dataSources.add(fds);
       return new SourceSwitchingIterator(fds);
     }
@@ -561,11 +555,12 @@
         throw new IllegalStateException();
       }
 
-      List<String> files = dataSources.stream().map(x -> x.file).collect(Collectors.toList());
-      Map<FileSKVIterator,String> newlyReservedReaders = openFiles(files);
-      Map<String,List<FileSKVIterator>> map = new HashMap<>();
+      List<StoredTabletFile> files =
+          dataSources.stream().map(x -> x.file).collect(Collectors.toList());
+      Map<FileSKVIterator,StoredTabletFile> newlyReservedReaders = openFiles(files);
+      Map<StoredTabletFile,List<FileSKVIterator>> map = new HashMap<>();
       newlyReservedReaders.forEach(
-          (reader, fileName) -> map.computeIfAbsent(fileName, k -> new LinkedList<>()).add(reader));
+          (reader, file) -> map.computeIfAbsent(file, k -> new LinkedList<>()).add(reader));
 
       for (FileDataSource fds : dataSources) {
         FileSKVIterator source = map.get(fds.file).remove(0);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
deleted file mode 100644
index fc0404d..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.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.server.fs;
-
-import org.apache.accumulo.core.spi.fs.DelegatingChooser;
-import org.slf4j.LoggerFactory;
-
-@Deprecated(since = "2.1.0")
-public class PerTableVolumeChooser extends DelegatingChooser implements VolumeChooser {
-  public PerTableVolumeChooser() {
-    LoggerFactory.getLogger(PerTableVolumeChooser.class).warn(
-        "The class {} is deprecated.  Please configure {} instead.",
-        PerTableVolumeChooser.class.getName(), DelegatingChooser.class.getName());
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
deleted file mode 100644
index 5998960..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
+++ /dev/null
@@ -1,36 +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.server.fs;
-
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Same name used for compatibility during deprecation cycle")
-public class PreferredVolumeChooser extends org.apache.accumulo.core.spi.fs.PreferredVolumeChooser
-    implements VolumeChooser {
-  public PreferredVolumeChooser() {
-    LoggerFactory.getLogger(PreferredVolumeChooser.class).warn(
-        "The class {} is deprecated.  Please configure {} instead.",
-        PreferredVolumeChooser.class.getName(),
-        org.apache.accumulo.core.spi.fs.PreferredVolumeChooser.class.getName());
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
deleted file mode 100644
index 56cf8cc..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
+++ /dev/null
@@ -1,36 +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.server.fs;
-
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Same name used for compatibility during deprecation cycle")
-public class RandomVolumeChooser extends org.apache.accumulo.core.spi.fs.RandomVolumeChooser
-    implements VolumeChooser {
-  public RandomVolumeChooser() {
-    LoggerFactory.getLogger(RandomVolumeChooser.class).warn(
-        "The class {} is deprecated.  Please configure {} instead.",
-        RandomVolumeChooser.class.getName(),
-        org.apache.accumulo.core.spi.fs.RandomVolumeChooser.class.getName());
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
deleted file mode 100644
index 29711f0..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
+++ /dev/null
@@ -1,36 +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.server.fs;
-
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Same name used for compatibility during deprecation cycle")
-public class SpaceAwareVolumeChooser extends org.apache.accumulo.core.spi.fs.SpaceAwareVolumeChooser
-    implements VolumeChooser {
-  public SpaceAwareVolumeChooser() {
-    LoggerFactory.getLogger(SpaceAwareVolumeChooser.class).warn(
-        "The class {} is deprecated.  Please configure {} instead.",
-        SpaceAwareVolumeChooser.class.getName(),
-        org.apache.accumulo.core.spi.fs.SpaceAwareVolumeChooser.class.getName());
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java
deleted file mode 100644
index ddc1360..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java
+++ /dev/null
@@ -1,87 +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.server.fs;
-
-import java.util.Set;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * @deprecated since 2.1.0; implement {@link org.apache.accumulo.core.spi.fs.VolumeChooser} instead.
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
-    justification = "Same name used for compatibility during deprecation cycle")
-public interface VolumeChooser extends org.apache.accumulo.core.spi.fs.VolumeChooser {
-
-  /**
-   * Choose a volume from the provided options.
-   *
-   * @param env the server environment provided by the calling framework
-   * @param options the list of volumes to choose from
-   * @return one of the options
-   * @throws VolumeChooserException if there is an error choosing (this is a RuntimeException); this
-   *         does not preclude other RuntimeExceptions from occurring
-   */
-  default String choose(VolumeChooserEnvironment env, String[] options)
-      throws VolumeChooserException {
-    throw new UnsupportedOperationException("This method will be removed in 3.0");
-  }
-
-  /**
-   * Default method provided for compatibility with 2.0.0.
-   *
-   * @since 2.1.0
-   */
-  @Override
-  default String choose(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
-      Set<String> options) {
-    InterfaceEvolutionWarner.warnOnce(getClass(), VolumeChooser.class,
-        "choose(VolumeChooserEnvironment,Set)", "3.0");
-    return choose((VolumeChooserEnvironmentImpl) env, options.toArray(new String[0]));
-  }
-
-  /**
-   * Default method provided for compatibility with 2.0.0.
-   *
-   * @since 2.1.0
-   */
-  @Override
-  default Set<String> choosable(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
-      Set<String> options) {
-    // assume that all options are possible to be chosen by this chooser
-    return options;
-  }
-
-  @Deprecated(since = "2.1.0")
-  class VolumeChooserException extends RuntimeException {
-
-    private static final long serialVersionUID = 1L;
-
-    public VolumeChooserException(String message) {
-      super(message);
-    }
-
-    public VolumeChooserException(String message, Throwable cause) {
-      super(message, cause);
-    }
-
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
deleted file mode 100644
index 8569e6b..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.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.server.fs;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
-    justification = "Same name used for compatibility during deprecation cycle")
-public interface VolumeChooserEnvironment
-    extends org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment {
-
-  /**
-   * A scope the volume chooser environment; a TABLE scope should be accompanied by a tableId.
-   *
-   * @since 2.0.0
-   */
-  public static enum ChooserScope {
-    DEFAULT, TABLE, INIT, LOGGER
-  }
-
-  /**
-   * The end row of the tablet for which a volume is being chosen. Only call this when the scope is
-   * TABLE
-   *
-   * @since 2.0.0
-   */
-  @Override
-  public Text getEndRow();
-
-  public boolean hasTableId();
-
-  public TableId getTableId();
-
-  /**
-   * @since 2.0.0
-   */
-  public default ChooserScope getScope() {
-
-    var scope = getChooserScope();
-    switch (scope) {
-      case DEFAULT:
-        return ChooserScope.DEFAULT;
-      case INIT:
-        return ChooserScope.INIT;
-      case LOGGER:
-        return ChooserScope.LOGGER;
-      case TABLE:
-        return ChooserScope.TABLE;
-      default:
-        throw new IllegalArgumentException("Unknown chooser scope : " + scope);
-    }
-  }
-
-  /**
-   * @since 2.0.0
-   */
-  @Override
-  public ServiceEnvironment getServiceEnv();
-
-  /**
-   * @since 2.0.0
-   */
-  public FileSystem getFileSystem(String option);
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
index 7b3f2e2..b07fd0f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
@@ -23,10 +23,9 @@
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServiceEnvironmentImpl;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -35,17 +34,14 @@
  * {@code VolumeChooserEnvironment} should result in more stable code over time than using this
  * class.
  */
-@SuppressWarnings("deprecation")
 public class VolumeChooserEnvironmentImpl implements VolumeChooserEnvironment {
 
-  private final ServerContext context;
   private final Scope scope;
   private final Optional<TableId> tableId;
   private final Text endRow;
   private final ServiceEnvironment senv;
 
   public VolumeChooserEnvironmentImpl(Scope scope, ServerContext context) {
-    this.context = context;
     this.scope = Objects.requireNonNull(scope);
     this.tableId = Optional.empty();
     this.endRow = null;
@@ -53,7 +49,6 @@
   }
 
   public VolumeChooserEnvironmentImpl(TableId tableId, Text endRow, ServerContext context) {
-    this.context = context;
     this.scope = Scope.TABLE;
     this.tableId = Optional.of(tableId);
     this.endRow = endRow;
@@ -62,7 +57,6 @@
 
   public VolumeChooserEnvironmentImpl(Scope scope, TableId tableId, Text endRow,
       ServerContext context) {
-    this.context = context;
     this.scope = Objects.requireNonNull(scope);
     this.tableId = Optional.of(tableId);
     this.endRow = endRow;
@@ -89,16 +83,6 @@
   }
 
   @Override
-  public boolean hasTableId() {
-    return tableId.isPresent();
-  }
-
-  @Override
-  public TableId getTableId() {
-    return tableId.orElseThrow();
-  }
-
-  @Override
   public Scope getChooserScope() {
     return this.scope;
   }
@@ -109,11 +93,6 @@
   }
 
   @Override
-  public FileSystem getFileSystem(String option) {
-    return context.getVolumeManager().getFileSystemByPath(new Path(option));
-  }
-
-  @Override
   public boolean equals(Object obj) {
     if (obj == this) {
       return true;
@@ -123,7 +102,7 @@
     }
     VolumeChooserEnvironmentImpl other = (VolumeChooserEnvironmentImpl) obj;
     return getChooserScope() == other.getChooserScope()
-        && Objects.equals(getTableId(), other.getTableId());
+        && Objects.equals(tableId.orElseThrow(), other.getTable().orElseThrow());
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
index a797506..018d473 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
@@ -20,6 +20,7 @@
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.Map;
@@ -218,18 +219,18 @@
       log.debug("Trying to read instance id from {}", instanceDirectory);
       if (files == null || files.length == 0) {
         log.error("unable to obtain instance id at {}", instanceDirectory);
-        throw new RuntimeException(
+        throw new IllegalStateException(
             "Accumulo not initialized, there is no instance id at " + instanceDirectory);
       } else if (files.length != 1) {
         log.error("multiple potential instances in {}", instanceDirectory);
-        throw new RuntimeException(
+        throw new IllegalStateException(
             "Accumulo found multiple possible instance ids in " + instanceDirectory);
       } else {
         return InstanceId.of(files[0].getPath().getName());
       }
     } catch (IOException e) {
       log.error("Problem reading instance id out of hdfs at " + instanceDirectory, e);
-      throw new RuntimeException(
+      throw new UncheckedIOException(
           "Can't tell if Accumulo is initialized; can't read instance id at " + instanceDirectory,
           e);
     } catch (IllegalArgumentException exception) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 0fa7d7b..40cdb58 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -104,7 +104,7 @@
       // null chooser handled below
     }
     if (chooser1 == null) {
-      throw new RuntimeException(
+      throw new IllegalStateException(
           "Failed to load volume chooser specified by " + Property.GENERAL_VOLUME_CHOOSER);
     }
     chooser = chooser1;
@@ -228,7 +228,7 @@
               + " not be configured as false. " + ticketMessage;
           // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility
           log.error("FATAL {}", msg);
-          throw new RuntimeException(msg);
+          throw new IllegalStateException(msg);
         }
 
         // Warn if synconclose isn't set
@@ -506,7 +506,7 @@
     if (!options.contains(choice)) {
       String msg = "The configured volume chooser, '" + chooser.getClass()
           + "', or one of its delegates returned a volume not in the set of options provided";
-      throw new RuntimeException(msg);
+      throw new IllegalStateException(msg);
     }
     return choice;
   }
@@ -519,7 +519,7 @@
       if (!options.contains(choice)) {
         String msg = "The configured volume chooser, '" + chooser.getClass()
             + "', or one of its delegates returned a volume not in the set of options provided";
-        throw new RuntimeException(msg);
+        throw new IllegalStateException(msg);
       }
     }
     return choices;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
index 35e1fbe..a994d16 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
@@ -18,25 +18,24 @@
  */
 package org.apache.accumulo.server.fs;
 
+import static java.util.Objects.requireNonNull;
+
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock;
+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.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,60 +55,34 @@
   }
 
   public static Path removeTrailingSlash(Path path) {
-    String pathStr = path.toString();
+    String pathStr = requireNonNull(path).toString();
     if (pathStr.endsWith("/")) {
       return new Path(removeTrailingSlash(pathStr));
     }
     return path;
   }
 
-  public static Path switchVolume(String path, FileType ft, List<Pair<Path,Path>> replacements) {
-    if (replacements.isEmpty()) {
-      log.trace("Not switching volume because there are no replacements");
-      return null;
+  public static Path switchVolume(Path path, FileType ft, Map<Path,Path> replacements) {
+    Path replacement = null;
+    if (!replacements.isEmpty()) {
+      // removing trailing slash for exact match comparison on the volume itself
+      Path volume = removeTrailingSlash(ft.getVolume(requireNonNull(path)));
+      replacement = replacements.entrySet().stream()
+          .filter(entry -> removeTrailingSlash(entry.getKey()).equals(volume))
+          .map(entry -> new Path(entry.getValue(), requireNonNull(ft.removeVolume(path))))
+          .findFirst().orElse(null);
     }
-    Path p = new Path(path);
-
-    // removing slash because new Path("hdfs://nn1").equals(new Path("hdfs://nn1/")) evaluates to
-    // false
-    Path volume = removeTrailingSlash(ft.getVolume(p));
-
-    for (Pair<Path,Path> pair : replacements) {
-      Path key = removeTrailingSlash(pair.getFirst());
-
-      if (key.equals(volume)) {
-        Path replacement = new Path(pair.getSecond(), ft.removeVolume(p));
-        log.trace("Replacing {} with {}", path, replacement);
-        return replacement;
-      }
+    if (replacement != null) {
+      log.trace("Replacing {} with {} for {}", path, replacement, ft);
+      return replacement;
     }
-
-    log.trace("Could not find replacement for {} at {}", ft, path);
-
+    log.trace("No replacement available for {} at {}", ft, path);
     return null;
   }
 
-  protected static LogEntry switchVolumes(LogEntry le, List<Pair<Path,Path>> replacements) {
-    Path switchedPath = switchVolume(le.filename, FileType.WAL, replacements);
-    String switchedString;
-    int numSwitched = 0;
-    if (switchedPath != null) {
-      switchedString = switchedPath.toString();
-      numSwitched++;
-    } else {
-      switchedString = le.filename;
-    }
-
-    if (numSwitched == 0) {
-      log.trace("Did not switch {}", le);
-      return null;
-    }
-
-    LogEntry newLogEntry = le.switchFile(switchedString);
-
-    log.trace("Switched {} to {}", le, newLogEntry);
-
-    return newLogEntry;
+  public static LogEntry switchVolume(LogEntry le, Map<Path,Path> replacements) {
+    Path switchedPath = switchVolume(new Path(le.getPath()), FileType.WAL, replacements);
+    return switchedPath == null ? null : LogEntry.fromPath(switchedPath.toString());
   }
 
   public static class TabletFiles {
@@ -136,8 +109,8 @@
    * for use it chooses a new tablet directory.
    */
   public static TabletFiles updateTabletVolumes(ServerContext context, ServiceLock zooLock,
-      KeyExtent extent, TabletFiles tabletFiles, boolean replicate) {
-    List<Pair<Path,Path>> replacements = context.getVolumeReplacements();
+      KeyExtent extent, TabletFiles tabletFiles) {
+    Map<Path,Path> replacements = context.getVolumeReplacements();
     if (replacements.isEmpty()) {
       return tabletFiles;
     }
@@ -147,29 +120,30 @@
     List<LogEntry> logsToAdd = new ArrayList<>();
 
     List<StoredTabletFile> filesToRemove = new ArrayList<>();
-    SortedMap<TabletFile,DataFileValue> filesToAdd = new TreeMap<>();
+    SortedMap<ReferencedTabletFile,DataFileValue> filesToAdd = new TreeMap<>();
 
     TabletFiles ret = new TabletFiles();
 
     for (LogEntry logEntry : tabletFiles.logEntries) {
-      LogEntry switchedLogEntry = switchVolumes(logEntry, replacements);
+      LogEntry switchedLogEntry = switchVolume(logEntry, replacements);
       if (switchedLogEntry != null) {
         logsToRemove.add(logEntry);
         logsToAdd.add(switchedLogEntry);
         ret.logEntries.add(switchedLogEntry);
-        log.debug("Replacing volume {} : {} -> {}", extent, logEntry.filename,
-            switchedLogEntry.filename);
+        log.debug("Replacing volume {} : {} -> {}", extent, logEntry.getPath(),
+            switchedLogEntry.getPath());
       } else {
         ret.logEntries.add(logEntry);
       }
     }
 
     for (Entry<StoredTabletFile,DataFileValue> entry : tabletFiles.datafiles.entrySet()) {
-      String metaPath = entry.getKey().getMetaUpdateDelete();
-      Path switchedPath = switchVolume(metaPath, FileType.TABLE, replacements);
+      String metaPath = entry.getKey().getMetadata();
+      Path switchedPath = switchVolume(entry.getKey().getPath(), FileType.TABLE, replacements);
       if (switchedPath != null) {
         filesToRemove.add(entry.getKey());
-        TabletFile switchedFile = new TabletFile(switchedPath);
+        ReferencedTabletFile switchedFile =
+            new ReferencedTabletFile(switchedPath, entry.getKey().getRange());
         filesToAdd.put(switchedFile, entry.getValue());
         ret.datafiles.put(switchedFile.insert(), entry.getValue());
         log.debug("Replacing volume {} : {} -> {}", extent, metaPath, switchedPath);
@@ -181,16 +155,6 @@
     if (logsToRemove.size() + filesToRemove.size() > 0) {
       MetadataTableUtil.updateTabletVolumes(extent, logsToRemove, logsToAdd, filesToRemove,
           filesToAdd, zooLock, context);
-      if (replicate) {
-        @SuppressWarnings("deprecation")
-        Status status = org.apache.accumulo.server.replication.StatusUtil.fileClosed();
-        log.debug("Tablet directory switched, need to record old log files {} {}", logsToRemove,
-            ProtobufUtil.toString(status));
-        // Before deleting these logs, we need to mark them for replication
-        for (LogEntry logEntry : logsToRemove) {
-          ReplicationTableUtil.updateFiles(context, extent, logEntry.filename, status);
-        }
-      }
     }
 
     // method this should return the exact strings that are in the metadata table
diff --git a/server/base/src/main/java/org/apache/accumulo/server/gc/AllVolumesDirectory.java b/server/base/src/main/java/org/apache/accumulo/server/gc/AllVolumesDirectory.java
index aff8dd5..df45a16 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/gc/AllVolumesDirectory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/gc/AllVolumesDirectory.java
@@ -42,8 +42,8 @@
   }
 
   @Override
-  public String getMetadataEntry() {
-    return metadataEntry;
+  public String getMetadataPath() {
+    return metadataPath;
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
index 7e99f13..8fdfa92 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/FileSystemInitializer.java
@@ -21,7 +21,6 @@
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN;
-import static org.apache.accumulo.server.init.Initialize.REPL_TABLE_ID;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -32,7 +31,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
 import org.apache.accumulo.core.data.InstanceId;
@@ -42,8 +40,8 @@
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
-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.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataTime;
@@ -97,43 +95,27 @@
     Text splitPoint = MetadataSchema.TabletsSection.getRange().getEndKey().getRow();
 
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(
-        VolumeChooserEnvironment.Scope.INIT, MetadataTable.ID, splitPoint, context);
+        VolumeChooserEnvironment.Scope.INIT, AccumuloTable.METADATA.tableId(), splitPoint, context);
     String tableMetadataTabletDirName = TABLE_TABLETS_TABLET_DIR;
     String tableMetadataTabletDirUri =
         fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
-            + MetadataTable.ID + Path.SEPARATOR + tableMetadataTabletDirName;
+            + AccumuloTable.METADATA.tableId() + Path.SEPARATOR + tableMetadataTabletDirName;
     chooserEnv = new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT,
-        REPL_TABLE_ID, null, context);
-    String replicationTableDefaultTabletDirName =
-        MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
-    String replicationTableDefaultTabletDirUri =
-        fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
-            + REPL_TABLE_ID + Path.SEPARATOR + replicationTableDefaultTabletDirName;
-    chooserEnv = new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.INIT,
-        MetadataTable.ID, null, context);
+        AccumuloTable.METADATA.tableId(), null, context);
     String defaultMetadataTabletDirName =
         MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
     String defaultMetadataTabletDirUri =
         fs.choose(chooserEnv, context.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
-            + MetadataTable.ID + Path.SEPARATOR + defaultMetadataTabletDirName;
+            + AccumuloTable.METADATA.tableId() + Path.SEPARATOR + defaultMetadataTabletDirName;
 
     // create table and default tablets directories
-    createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, defaultMetadataTabletDirUri,
-        replicationTableDefaultTabletDirUri);
-
-    String ext = FileOperations.getNewFileExtension(DefaultConfiguration.getInstance());
-
-    // populate the metadata tablet with info about the replication tablet
-    String metadataFileName = tableMetadataTabletDirUri + Path.SEPARATOR + "0_1." + ext;
-    Tablet replicationTablet =
-        new Tablet(REPL_TABLE_ID, replicationTableDefaultTabletDirName, null, null);
-    createMetadataFile(fs, metadataFileName, siteConfig, replicationTablet);
+    createDirectories(fs, rootTabletDirUri, tableMetadataTabletDirUri, defaultMetadataTabletDirUri);
 
     // populate the root tablet with info about the metadata table's two initial tablets
-    Tablet tablesTablet = new Tablet(MetadataTable.ID, tableMetadataTabletDirName, null, splitPoint,
-        metadataFileName);
-    Tablet defaultTablet =
-        new Tablet(MetadataTable.ID, defaultMetadataTabletDirName, splitPoint, null);
+    Tablet tablesTablet =
+        new Tablet(AccumuloTable.METADATA.tableId(), tableMetadataTabletDirName, null, splitPoint);
+    Tablet defaultTablet = new Tablet(AccumuloTable.METADATA.tableId(),
+        defaultMetadataTabletDirName, splitPoint, null);
     createMetadataFile(fs, rootTabletFileUri, siteConfig, tablesTablet, defaultTablet);
   }
 
@@ -158,11 +140,10 @@
 
   private void initSystemTablesConfig(final ServerContext context)
       throws IOException, InterruptedException, KeeperException {
-    setTableProperties(context, RootTable.ID, initConfig.getRootTableConf());
-    setTableProperties(context, RootTable.ID, initConfig.getRootMetaConf());
-    setTableProperties(context, MetadataTable.ID, initConfig.getRootMetaConf());
-    setTableProperties(context, MetadataTable.ID, initConfig.getMetaTableConf());
-    setTableProperties(context, REPL_TABLE_ID, initConfig.getReplTableConf());
+    setTableProperties(context, AccumuloTable.ROOT.tableId(), initConfig.getRootTableConf());
+    setTableProperties(context, AccumuloTable.ROOT.tableId(), initConfig.getRootMetaConf());
+    setTableProperties(context, AccumuloTable.METADATA.tableId(), initConfig.getRootMetaConf());
+    setTableProperties(context, AccumuloTable.METADATA.tableId(), initConfig.getMetaTableConf());
   }
 
   private void setTableProperties(final ServerContext context, TableId tableId,
@@ -183,12 +164,13 @@
     for (Tablet tablet : tablets) {
       createEntriesForTablet(sorted, tablet);
     }
-    FileSystem fs = volmanager.getFileSystemByPath(new Path(fileName));
+    ReferencedTabletFile file = ReferencedTabletFile.of(new Path(fileName));
+    FileSystem fs = volmanager.getFileSystemByPath(file.getPath());
 
     CryptoService cs = CryptoFactoryLoader.getServiceForServer(conf);
 
     FileSKVWriter tabletWriter = FileOperations.getInstance().newWriterBuilder()
-        .forFile(fileName, fs, fs.getConf(), cs).withTableConfiguration(conf).build();
+        .forFile(file, fs, fs.getConf(), cs).withTableConfiguration(conf).build();
     tabletWriter.startDefaultLocalityGroup();
 
     for (Map.Entry<Key,Value> entry : sorted.entrySet()) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
index eedbca9..77773b1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/InitialConfiguration.java
@@ -18,32 +18,21 @@
  */
 package org.apache.accumulo.server.init;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.function.Predicate;
 
-import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.iterators.Combiner;
-import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
-import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.server.constraints.MetadataConstraints;
 import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-
-import com.google.common.base.Joiner;
 
 class InitialConfiguration {
 
@@ -53,7 +42,6 @@
   private final HashMap<String,String> initialRootMetaConf = new HashMap<>();
   // config for only metadata table
   private final HashMap<String,String> initialMetaConf = new HashMap<>();
-  private final HashMap<String,String> initialReplicationTableConf = new HashMap<>();
   private final Configuration hadoopConf;
   private final SiteConfiguration siteConf;
 
@@ -103,64 +91,6 @@
         SimpleCompactionDispatcher.class.getName());
     initialMetaConf.put(Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", "meta");
 
-    // ACCUMULO-3077 Set the combiner on accumulo.metadata during init to reduce the likelihood of a
-    // race condition where a tserver compacts away Status updates because it didn't see the
-    // Combiner
-    // configured
-    @SuppressWarnings("deprecation")
-    var statusCombinerClass = org.apache.accumulo.server.replication.StatusCombiner.class;
-    IteratorSetting setting =
-        new IteratorSetting(9, ReplicationTableUtil.COMBINER_NAME, statusCombinerClass);
-    Combiner.setColumns(setting, Collections
-        .singletonList(new IteratorSetting.Column(MetadataSchema.ReplicationSection.COLF)));
-    for (IteratorUtil.IteratorScope scope : IteratorUtil.IteratorScope.values()) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX,
-          scope.name().toLowerCase(), setting.getName());
-      for (Map.Entry<String,String> prop : setting.getOptions().entrySet()) {
-        initialMetaConf.put(root + ".opt." + prop.getKey(), prop.getValue());
-      }
-      initialMetaConf.put(root, setting.getPriority() + "," + setting.getIteratorClass());
-    }
-
-    // add combiners to replication table
-    @SuppressWarnings("deprecation")
-    String replicationCombinerName =
-        org.apache.accumulo.core.replication.ReplicationTable.COMBINER_NAME;
-    setting = new IteratorSetting(30, replicationCombinerName, statusCombinerClass);
-    setting.setPriority(30);
-    @SuppressWarnings("deprecation")
-    Text statusSectionName =
-        org.apache.accumulo.core.replication.ReplicationSchema.StatusSection.NAME;
-    @SuppressWarnings("deprecation")
-    Text workSectionName = org.apache.accumulo.core.replication.ReplicationSchema.WorkSection.NAME;
-    Combiner.setColumns(setting, Arrays.asList(new IteratorSetting.Column(statusSectionName),
-        new IteratorSetting.Column(workSectionName)));
-    for (IteratorUtil.IteratorScope scope : EnumSet.allOf(IteratorUtil.IteratorScope.class)) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX,
-          scope.name().toLowerCase(), setting.getName());
-      for (Map.Entry<String,String> prop : setting.getOptions().entrySet()) {
-        initialReplicationTableConf.put(root + ".opt." + prop.getKey(), prop.getValue());
-      }
-      initialReplicationTableConf.put(root,
-          setting.getPriority() + "," + setting.getIteratorClass());
-    }
-    // add locality groups to replication table
-    @SuppressWarnings("deprecation")
-    Map<String,Set<Text>> replicationLocalityGroups =
-        org.apache.accumulo.core.replication.ReplicationTable.LOCALITY_GROUPS;
-    for (Map.Entry<String,Set<Text>> g : replicationLocalityGroups.entrySet()) {
-      initialReplicationTableConf.put(Property.TABLE_LOCALITY_GROUP_PREFIX + g.getKey(),
-          LocalityGroupUtil.encodeColumnFamilies(g.getValue()));
-    }
-    initialReplicationTableConf.put(Property.TABLE_LOCALITY_GROUPS.getKey(),
-        Joiner.on(",").join(replicationLocalityGroups.keySet()));
-    // add formatter to replication table
-    @SuppressWarnings("deprecation")
-    String replicationFormatterClassName =
-        org.apache.accumulo.server.replication.ReplicationUtil.STATUS_FORMATTER_CLASS_NAME;
-    initialReplicationTableConf.put(Property.TABLE_FORMATTER_CLASS.getKey(),
-        replicationFormatterClassName);
-
     int max = hadoopConf.getInt("dfs.replication.max", 512);
     // Hadoop 0.23 switched the min value configuration name
     int min = Math.max(hadoopConf.getInt("dfs.replication.min", 1),
@@ -176,8 +106,9 @@
   private void setMetadataReplication(int replication, String reason) {
     String rep = System.console()
         .readLine("Your HDFS replication " + reason + " is not compatible with our default "
-            + MetadataTable.NAME + " replication of 5. What do you want to set your "
-            + MetadataTable.NAME + " replication to? (" + replication + ") ");
+            + AccumuloTable.METADATA.tableName()
+            + " replication of 5. What do you want to set your "
+            + AccumuloTable.METADATA.tableName() + " replication to? (" + replication + ") ");
     if (rep == null || rep.isEmpty()) {
       rep = Integer.toString(replication);
     } else {
@@ -199,10 +130,6 @@
     return initialMetaConf;
   }
 
-  HashMap<String,String> getReplTableConf() {
-    return initialReplicationTableConf;
-  }
-
   Configuration getHadoopConf() {
     return hadoopConf;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 800dcc3..8a6716d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -39,14 +39,13 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
-import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.server.AccumuloDataVersion;
 import org.apache.accumulo.server.ServerContext;
@@ -84,8 +83,6 @@
 
   private static final Logger log = LoggerFactory.getLogger(Initialize.class);
   private static final String DEFAULT_ROOT_USER = "root";
-  @SuppressWarnings("deprecation")
-  static final TableId REPL_TABLE_ID = org.apache.accumulo.core.replication.ReplicationTable.ID;
 
   static void checkInit(ZooReaderWriter zoo, VolumeManager fs, InitialConfiguration initConfig)
       throws IOException {
@@ -165,13 +162,13 @@
 
     try (ServerContext context =
         ServerContext.initialize(initConfig.getSiteConf(), instanceName, instanceId)) {
-      var chooserEnv = new VolumeChooserEnvironmentImpl(Scope.INIT, RootTable.ID, null, context);
+      var chooserEnv =
+          new VolumeChooserEnvironmentImpl(Scope.INIT, AccumuloTable.ROOT.tableId(), null, context);
       String rootTabletDirName = RootTable.ROOT_TABLET_DIR_NAME;
       String ext = FileOperations.getNewFileExtension(DefaultConfiguration.getInstance());
-      String rootTabletFileUri = new Path(
-          fs.choose(chooserEnv, initConfig.getVolumeUris()) + SEPARATOR + TABLE_DIR + SEPARATOR
-              + RootTable.ID + SEPARATOR + rootTabletDirName + SEPARATOR + "00000_00000." + ext)
-          .toString();
+      String rootTabletFileUri = new Path(fs.choose(chooserEnv, initConfig.getVolumeUris())
+          + SEPARATOR + TABLE_DIR + SEPARATOR + AccumuloTable.ROOT.tableId() + SEPARATOR
+          + rootTabletDirName + SEPARATOR + "00000_00000." + ext).toString();
       zki.initialize(context, opts.clearInstanceName, instanceNamePath, rootTabletDirName,
           rootTabletFileUri);
 
@@ -180,8 +177,8 @@
       }
       var fileSystemInitializer = new FileSystemInitializer(initConfig, zoo, instanceId);
       var rootVol = fs.choose(chooserEnv, initConfig.getVolumeUris());
-      var rootPath = new Path(rootVol + SEPARATOR + TABLE_DIR + SEPARATOR + RootTable.ID + SEPARATOR
-          + rootTabletDirName);
+      var rootPath = new Path(rootVol + SEPARATOR + TABLE_DIR + SEPARATOR
+          + AccumuloTable.ROOT.tableId() + SEPARATOR + rootTabletDirName);
       fileSystemInitializer.initialize(fs, rootPath.toString(), rootTabletFileUri, context);
 
       checkSASL(initConfig);
@@ -456,8 +453,8 @@
     Path versionPath = new Path(aBasePath, Constants.VERSION_DIR);
 
     InstanceId instanceId = VolumeManager.getInstanceIDFromHdfs(iidPath, hadoopConf);
-    for (Pair<Path,Path> replacementVolume : serverDirs.getVolumeReplacements()) {
-      if (aBasePath.equals(replacementVolume.getFirst())) {
+    for (Path replacedVolume : serverDirs.getVolumeReplacements().keySet()) {
+      if (aBasePath.equals(replacedVolume)) {
         log.error(
             "{} is set to be replaced in {} and should not appear in {}."
                 + " It is highly recommended that this property be removed as data"
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
index 5a74a2d..e861f31 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.server.init;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.server.init.Initialize.REPL_TABLE_ID;
 
 import java.io.IOException;
 
@@ -33,8 +32,9 @@
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
-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.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataTime;
@@ -118,14 +118,12 @@
     TableManager.prepareNewNamespaceState(context, Namespace.ACCUMULO.id(),
         Namespace.ACCUMULO.name(), ZooUtil.NodeExistsPolicy.FAIL);
 
-    TableManager.prepareNewTableState(context, RootTable.ID, Namespace.ACCUMULO.id(),
-        RootTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(context, MetadataTable.ID, Namespace.ACCUMULO.id(),
-        MetadataTable.NAME, TableState.ONLINE, ZooUtil.NodeExistsPolicy.FAIL);
-    @SuppressWarnings("deprecation")
-    String replicationTableName = org.apache.accumulo.core.replication.ReplicationTable.NAME;
-    TableManager.prepareNewTableState(context, REPL_TABLE_ID, Namespace.ACCUMULO.id(),
-        replicationTableName, TableState.OFFLINE, ZooUtil.NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(context, AccumuloTable.ROOT.tableId(),
+        Namespace.ACCUMULO.id(), AccumuloTable.ROOT.tableName(), TableState.ONLINE,
+        ZooUtil.NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(context, AccumuloTable.METADATA.tableId(),
+        Namespace.ACCUMULO.id(), AccumuloTable.METADATA.tableName(), TableState.ONLINE,
+        ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, EMPTY_BYTE_ARRAY,
         ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, EMPTY_BYTE_ARRAY,
@@ -157,15 +155,6 @@
         ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMONITOR_LOCK, EMPTY_BYTE_ARRAY,
         ZooUtil.NodeExistsPolicy.FAIL);
-    @SuppressWarnings("deprecation")
-    String replicationZBase = org.apache.accumulo.core.replication.ReplicationConstants.ZOO_BASE;
-    zoo.putPersistentData(zkInstanceRoot + replicationZBase, EMPTY_BYTE_ARRAY,
-        ZooUtil.NodeExistsPolicy.FAIL);
-    @SuppressWarnings("deprecation")
-    String replicationZServers =
-        org.apache.accumulo.core.replication.ReplicationConstants.ZOO_TSERVERS;
-    zoo.putPersistentData(zkInstanceRoot + replicationZServers, EMPTY_BYTE_ARRAY,
-        ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + WalStateManager.ZWALS, EMPTY_BYTE_ARRAY,
         ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZCOORDINATOR, EMPTY_BYTE_ARRAY,
@@ -188,8 +177,8 @@
     MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mutation,
         new Value(dirName));
 
-    mutation.put(MetadataSchema.TabletsSection.DataFileColumnFamily.STR_NAME, file,
-        new DataFileValue(0, 0).encodeAsValue());
+    mutation.put(MetadataSchema.TabletsSection.DataFileColumnFamily.STR_NAME,
+        StoredTabletFile.serialize(file), new DataFileValue(0, 0).encodeAsValue());
 
     MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mutation,
         new Value(new MetadataTime(0, TimeType.LOGICAL).encode()));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java
index 72daff6..e13fa93 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/iterators/SystemIteratorEnvironment.java
@@ -30,4 +30,9 @@
 
   SortedKeyValueIterator<Key,Value> getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter);
 
+  @Override
+  default boolean isRunningLowOnMemory() {
+    return getServerContext().getLowMemoryDetector().isRunningLowOnMemory();
+  }
+
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java
index 63970b2..040078b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/iterators/TabletIteratorEnvironment.java
@@ -18,11 +18,10 @@
  */
 package org.apache.accumulo.server.iterators;
 
-import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Map;
 
+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;
@@ -33,7 +32,7 @@
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
@@ -42,7 +41,6 @@
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServiceEnvironmentImpl;
 import org.apache.accumulo.server.fs.FileManager.ScanFileManager;
-import org.apache.hadoop.fs.Path;
 
 public class TabletIteratorEnvironment implements SystemIteratorEnvironment {
 
@@ -55,7 +53,7 @@
   private final AccumuloConfiguration tableConfig;
   private final TableId tableId;
   private final ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators;
-  private Map<TabletFile,DataFileValue> files;
+  private Map<StoredTabletFile,DataFileValue> files;
 
   private final Authorizations authorizations; // these will only be supplied during scan scope
   private SamplerConfiguration samplerConfig;
@@ -81,7 +79,7 @@
 
   public TabletIteratorEnvironment(ServerContext context, IteratorScope scope,
       AccumuloConfiguration tableConfig, TableId tableId, ScanFileManager trm,
-      Map<TabletFile,DataFileValue> files, Authorizations authorizations,
+      Map<StoredTabletFile,DataFileValue> files, Authorizations authorizations,
       SamplerConfigurationImpl samplerConfig,
       ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators) {
     if (scope == IteratorScope.majc) {
@@ -126,12 +124,6 @@
     this.topLevelIterators = new ArrayList<>();
   }
 
-  @Deprecated(since = "2.0.0")
-  @Override
-  public AccumuloConfiguration getConfig() {
-    return tableConfig;
-  }
-
   @Override
   public IteratorScope getIteratorScope() {
     return scope;
@@ -154,20 +146,6 @@
     return userCompaction;
   }
 
-  @Deprecated(since = "2.0.0")
-  @Override
-  public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName)
-      throws IOException {
-    TabletFile ref = new TabletFile(new Path(mapFileName));
-    return trm.openFiles(Collections.singletonMap(ref, files.get(ref)), false, null).get(0);
-  }
-
-  @Deprecated(since = "2.0.0")
-  @Override
-  public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
-    topLevelIterators.add(iter);
-  }
-
   @Override
   public Authorizations getAuthorizations() {
     if (scope != IteratorScope.scan) {
@@ -226,9 +204,8 @@
     return context;
   }
 
-  @Deprecated(since = "2.1.0")
   @Override
-  public ServiceEnvironment getServiceEnv() {
+  public PluginEnvironment getPluginEnv() {
     return serviceEnvironment;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
index 4ad40e0..b2cc933 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
@@ -41,12 +41,11 @@
 
 /**
  * This class governs the space in Zookeeper that advertises the status of Write-Ahead Logs in use
- * by tablet servers and the replication machinery.
+ * by tablet servers.
  *
  * <p>
  * The Accumulo Manager needs to know the state of the WALs to mark tablets during recovery. The GC
- * needs to know when a log is no longer needed so it can be removed. The replication mechanism
- * needs to know when a log is closed and can be forwarded to the destination table.
+ * needs to know when a log is no longer needed so it can be removed.
  *
  * <p>
  * The state of the WALs is kept in Zookeeper under /accumulo/&lt;instanceid&gt;/wals. For each
@@ -63,9 +62,6 @@
  * update the tablets assigned to that server with log references. Once all tablets have been
  * reassigned and the log references are removed, the log will be eligible for deletion.
  *
- * <p>
- * Even when a log is UNREFERENCED by the tablet server, the replication mechanism may still need
- * the log. The GC will defer log removal until replication is finished with it.
  */
 public class WalStateManager {
 
@@ -256,7 +252,7 @@
     }
   }
 
-  // tablet server can mark the log as closed (but still needed), for replication to begin
+  // tablet server can mark the log as closed (but still needed)
   // manager can mark a log as unreferenced after it has made log recovery markers on the tablets
   // that need to be recovered
   public void closeWal(TServerInstance instance, Path path) throws WalMarkerException {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
index e91dcf9..b41c381 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.server.manager;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy.SKIP;
 
@@ -35,21 +34,21 @@
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 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.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
@@ -62,6 +61,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class LiveTServerSet implements Watcher {
 
   public interface Listener {
@@ -89,8 +90,8 @@
       return mlock.getLockID().serialize(context.getZooKeeperRoot() + Constants.ZMANAGER_LOCK);
     }
 
-    private void loadTablet(TabletClientService.Client client, ServiceLock lock, KeyExtent extent)
-        throws TException {
+    private void loadTablet(TabletManagementClientService.Client client, ServiceLock lock,
+        KeyExtent extent) throws TException {
       client.loadTablet(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock),
           extent.toThrift());
     }
@@ -99,13 +100,13 @@
       if (extent.isMeta()) {
         // see ACCUMULO-3597
         try (TTransport transport = ThriftUtil.createTransport(address, context)) {
-          TabletClientService.Client client =
-              ThriftUtil.createClient(ThriftClientTypes.TABLET_SERVER, transport);
+          TabletManagementClientService.Client client =
+              ThriftUtil.createClient(ThriftClientTypes.TABLET_MGMT, transport);
           loadTablet(client, lock, extent);
         }
       } else {
-        TabletClientService.Client client =
-            ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
+        TabletManagementClientService.Client client =
+            ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context);
         try {
           loadTablet(client, lock, extent);
         } finally {
@@ -116,8 +117,8 @@
 
     public void unloadTablet(ServiceLock lock, KeyExtent extent, TUnloadTabletGoal goal,
         long requestTime) throws TException {
-      TabletClientService.Client client =
-          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
+      TabletManagementClientService.Client client =
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context);
       try {
         client.unloadTablet(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock),
             extent.toThrift(), goal, requestTime);
@@ -136,7 +137,7 @@
       long start = System.currentTimeMillis();
 
       try (TTransport transport = ThriftUtil.createTransport(address, context)) {
-        TabletClientService.Client client =
+        TabletServerClientService.Client client =
             ThriftUtil.createClient(ThriftClientTypes.TABLET_SERVER, transport);
         TabletServerStatus status =
             client.getTabletServerStatus(TraceUtil.traceInfo(), context.rpcCreds());
@@ -148,7 +149,7 @@
     }
 
     public void halt(ServiceLock lock) throws TException, ThriftSecurityException {
-      TabletClientService.Client client =
+      TabletServerClientService.Client client =
           ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.halt(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock));
@@ -158,7 +159,7 @@
     }
 
     public void fastHalt(ServiceLock lock) throws TException {
-      TabletClientService.Client client =
+      TabletServerClientService.Client client =
           ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.fastHalt(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock));
@@ -169,7 +170,7 @@
 
     public void flush(ServiceLock lock, TableId tableId, byte[] startRow, byte[] endRow)
         throws TException {
-      TabletClientService.Client client =
+      TabletServerClientService.Client client =
           ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.flush(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock),
@@ -180,20 +181,10 @@
       }
     }
 
-    public void chop(ServiceLock lock, KeyExtent extent) throws TException {
-      TabletClientService.Client client =
-          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
-      try {
-        client.chop(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock), extent.toThrift());
-      } finally {
-        ThriftUtil.returnClient(client, context);
-      }
-    }
-
     public void splitTablet(KeyExtent extent, Text splitPoint)
         throws TException, ThriftSecurityException, NotServingTabletException {
-      TabletClientService.Client client =
-          ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
+      TabletManagementClientService.Client client =
+          ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context);
       try {
         client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(), extent.toThrift(),
             ByteBuffer.wrap(splitPoint.getBytes(), 0, splitPoint.getLength()));
@@ -204,7 +195,7 @@
 
     public void compact(ServiceLock lock, String tableId, byte[] startRow, byte[] endRow)
         throws TException {
-      TabletClientService.Client client =
+      TabletServerClientService.Client client =
           ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         client.compact(TraceUtil.traceInfo(), context.rpcCreds(), lockString(lock), tableId,
@@ -305,9 +296,10 @@
 
     final var zLockPath = ServiceLock.path(path + "/" + zPath);
     ZcStat stat = new ZcStat();
-    byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+    HostAndPort address = ServiceLock.getLockData(getZooCache(), zLockPath, stat)
+        .map(sld -> sld.getAddress(ServiceLockData.ThriftService.TSERV)).orElse(null);
 
-    if (lockData == null) {
+    if (address == null) {
       if (info != null) {
         doomed.add(info.instance);
         current.remove(zPath);
@@ -323,19 +315,17 @@
       }
     } else {
       locklessServers.remove(zPath);
-      ServerServices services = new ServerServices(new String(lockData, UTF_8));
-      HostAndPort client = services.getAddress(ServerServices.Service.TSERV_CLIENT);
-      TServerInstance instance = new TServerInstance(client, stat.getEphemeralOwner());
+      TServerInstance instance = new TServerInstance(address, stat.getEphemeralOwner());
 
       if (info == null) {
         updates.add(instance);
-        TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(client));
+        TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(address));
         current.put(zPath, tServerInfo);
         currentInstances.put(instance, tServerInfo);
       } else if (!info.instance.equals(instance)) {
         doomed.add(info.instance);
         updates.add(instance);
-        TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(client));
+        TServerInfo tServerInfo = new TServerInfo(instance, new TServerConnection(address));
         current.put(zPath, tServerInfo);
         currentInstances.remove(info.instance);
         currentInstances.put(instance, tServerInfo);
@@ -408,11 +398,11 @@
       if (index == -1) {
         throw new IllegalArgumentException("Could not parse tabletserver '" + tabletServer + "'");
       }
-      addr = AddressUtil.parseAddress(tabletServer.substring(0, index), false);
+      addr = AddressUtil.parseAddress(tabletServer.substring(0, index));
       // Strip off the last bracket
       sessionId = tabletServer.substring(index + 1, tabletServer.length() - 1);
     } else {
-      addr = AddressUtil.parseAddress(tabletServer, false);
+      addr = AddressUtil.parseAddress(tabletServer);
     }
     for (Entry<String,TServerInfo> entry : servers.entrySet()) {
       if (entry.getValue().instance.getHostAndPort().equals(addr)) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
index f44e290..3018ae1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/balancer/BalancerEnvironmentImpl.java
@@ -44,9 +44,8 @@
 import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
 import org.apache.accumulo.core.spi.balancer.data.TabletStatistics;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServiceEnvironmentImpl;
 import org.apache.thrift.TException;
@@ -54,6 +53,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class BalancerEnvironmentImpl extends ServiceEnvironmentImpl implements BalancerEnvironment {
   private static final Logger log = LoggerFactory.getLogger(BalancerEnvironmentImpl.class);
 
@@ -74,11 +75,13 @@
   @Override
   public Map<TabletId,TabletServerId> listTabletLocations(TableId tableId) {
     Map<TabletId,TabletServerId> tablets = new LinkedHashMap<>();
-    for (var tm : TabletsMetadata.builder(getContext()).forTable(tableId).fetch(LOCATION, PREV_ROW)
-        .build()) {
-      tablets.put(new TabletIdImpl(tm.getExtent()),
-          TabletServerIdImpl.fromThrift(Optional.ofNullable(tm.getLocation())
-              .map(TabletMetadata.Location::getServerInstance).orElse(null)));
+    try (TabletsMetadata tabletsMetadata =
+        TabletsMetadata.builder(getContext()).forTable(tableId).fetch(LOCATION, PREV_ROW).build()) {
+      for (var tm : tabletsMetadata) {
+        tablets.put(new TabletIdImpl(tm.getExtent()),
+            TabletServerIdImpl.fromThrift(Optional.ofNullable(tm.getLocation())
+                .map(TabletMetadata.Location::getServerInstance).orElse(null)));
+      }
     }
     return tablets;
   }
@@ -88,7 +91,8 @@
       TableId tableId) throws AccumuloException, AccumuloSecurityException {
     log.debug("Scanning tablet server {} for table {}", tabletServerId, tableId);
     try {
-      TabletClientService.Client client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER,
+      TabletServerClientService.Client client = ThriftUtil.getClient(
+          ThriftClientTypes.TABLET_SERVER,
           HostAndPort.fromParts(tabletServerId.getHost(), tabletServerId.getPort()), getContext());
       try {
         return client
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/recovery/RecoveryPath.java b/server/base/src/main/java/org/apache/accumulo/server/manager/recovery/RecoveryPath.java
index e739887..893867b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/recovery/RecoveryPath.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/recovery/RecoveryPath.java
@@ -30,19 +30,22 @@
       String uuid = walPath.getName();
       // drop uuid
       walPath = walPath.getParent();
-      // recovered 1.4 WALs won't have a server component
-      if (!walPath.getName().equals(FileType.WAL.getDirectory())) {
-        // drop server
-        walPath = walPath.getParent();
-      }
 
-      if (!walPath.getName().equals(FileType.WAL.getDirectory())) {
-        throw new IllegalArgumentException("Bad path " + walPath);
+      // expect and drop the server component
+      if (walPath.getName().equals(FileType.WAL.getDirectory())) {
+        throw new IllegalArgumentException("Bath path " + walPath + " (missing server component)");
       }
-
-      // drop wal
       walPath = walPath.getParent();
 
+      // expect and drop the wal component
+      if (!walPath.getName().equals(FileType.WAL.getDirectory())) {
+        throw new IllegalArgumentException(
+            "Bad path " + walPath + " (missing wal directory component)");
+      }
+      walPath = walPath.getParent();
+
+      // create new path in recovery directory that is a sibling to the wal directory (same volume),
+      // without the server component
       walPath = new Path(walPath, FileType.RECOVERY.getDirectory());
       walPath = new Path(walPath, uuid);
 
@@ -50,7 +53,6 @@
     }
 
     throw new IllegalArgumentException("Bad path " + walPath);
-
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java
index 84c6225..63b0006 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/LoggingTabletStateStore.java
@@ -21,15 +21,16 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.hadoop.fs.Path;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Wraps a tablet state store and logs important events.
  */
@@ -85,7 +86,7 @@
 
   @Override
   public void suspend(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      Map<TServerInstance,List<Path>> logsForDeadServers, SteadyTime suspensionTimestamp)
       throws DistributedStoreException {
     wrapped.suspend(tablets, logsForDeadServers, suspensionTimestamp);
 
@@ -99,8 +100,7 @@
       if (location != null) {
         server = location.getHostAndPort();
       }
-      TabletLogger.suspended(tls.extent, server, suspensionTimestamp, TimeUnit.MILLISECONDS,
-          logsForDeadServers.size());
+      TabletLogger.suspended(tls.extent, server, suspensionTimestamp, logsForDeadServers.size());
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeInfo.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeInfo.java
index e112800..510dc5f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeInfo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeInfo.java
@@ -82,15 +82,10 @@
   }
 
   public boolean needsToBeChopped(KeyExtent otherExtent) {
-    // During a delete, the block after the merge will be stretched to cover the deleted area.
-    // Therefore, it needs to be chopped
-    if (!otherExtent.tableId().equals(extent.tableId())) {
-      return false;
-    }
-    if (isDelete()) {
+    if (isDelete() && otherExtent.tableId().equals(extent.tableId())) {
       return otherExtent.prevEndRow() != null && otherExtent.prevEndRow().equals(extent.endRow());
     } else {
-      return this.extent.overlaps(otherExtent);
+      return false;
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java
index 7ed178c..1ab0c6d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java
@@ -28,24 +28,21 @@
    */
   STARTED,
   /**
-   * put all matching tablets online, split tablets if we are deleting
-   */
-  SPLITTING,
-  /**
-   * after the tablet server chops the file, it marks the metadata table with a chopped marker
-   */
-  WAITING_FOR_CHOPPED,
-  /**
    * when the number of chopped tablets in the range matches the number of online tablets in the
    * range, take the tablets offline
    */
   WAITING_FOR_OFFLINE,
   /**
-   * when the number of chopped, offline tablets equals the number of merge tablets, begin the
-   * metadata updates
+   * when the number of offline tablets equals the number of merge tablets, begin the metadata
+   * updates
    */
   MERGING,
   /**
+   * when the operation has finished metadata updates for merge. We can now remove the merged
+   * tablets and clear the MERGED marker. Not used for delete
+   */
+  MERGED,
+  /**
    * merge is complete, the resulting tablet can be brought online, remove the marker in zookeeper
    */
   COMPLETE
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
index 9af4438..acc37a2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java
@@ -23,7 +23,7 @@
 import java.util.Map;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample;
@@ -32,6 +32,7 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 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.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.hadoop.fs.Path;
 
@@ -53,7 +54,7 @@
   }
 
   MetaDataStateStore(DataLevel level, ClientContext context, CurrentState state) {
-    this(level, context, state, MetadataTable.NAME);
+    this(level, context, state, AccumuloTable.METADATA.tableName());
   }
 
   @Override
@@ -99,18 +100,18 @@
   @Override
   public void unassign(Collection<TabletLocationState> tablets,
       Map<TServerInstance,List<Path>> logsForDeadServers) throws DistributedStoreException {
-    unassign(tablets, logsForDeadServers, -1);
+    unassign(tablets, logsForDeadServers, null);
   }
 
   @Override
   public void suspend(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      Map<TServerInstance,List<Path>> logsForDeadServers, SteadyTime suspensionTimestamp)
       throws DistributedStoreException {
     unassign(tablets, logsForDeadServers, suspensionTimestamp);
   }
 
   private void unassign(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      Map<TServerInstance,List<Path>> logsForDeadServers, SteadyTime suspensionTimestamp)
       throws DistributedStoreException {
     try (var tabletsMutator = ample.mutateTablets()) {
       for (TabletLocationState tls : tablets) {
@@ -123,16 +124,16 @@
             List<Path> logs = logsForDeadServers.get(tls.current.getServerInstance());
             if (logs != null) {
               for (Path log : logs) {
-                LogEntry entry = new LogEntry(tls.extent, 0, log.toString());
+                LogEntry entry = LogEntry.fromPath(log.toString());
                 tabletMutator.putWal(entry);
               }
             }
           }
-          if (suspensionTimestamp >= 0) {
+          if (suspensionTimestamp != null && suspensionTimestamp.getMillis() >= 0) {
             tabletMutator.putSuspension(tls.current.getServerInstance(), suspensionTimestamp);
           }
         }
-        if (tls.suspend != null && suspensionTimestamp < 0) {
+        if (tls.suspend != null && suspensionTimestamp == null) {
           tabletMutator.deleteSuspension();
         }
         if (tls.hasFuture()) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
index dbb6113..b340c19 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
@@ -19,10 +19,9 @@
 package org.apache.accumulo.server.manager.state;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.lang.ref.Cleaner.Cleanable;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -45,7 +44,6 @@
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletLocationState.BadLocationStateException;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 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;
@@ -54,6 +52,7 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.cleaner.CleanerUtil;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -88,7 +87,6 @@
     scanner.fetchColumnFamily(LastLocationColumnFamily.NAME);
     scanner.fetchColumnFamily(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily());
     scanner.fetchColumnFamily(LogColumnFamily.NAME);
-    scanner.fetchColumnFamily(ChoppedColumnFamily.NAME);
     scanner.addScanIterator(new IteratorSetting(1000, "wholeRows", WholeRowIterator.class));
     IteratorSetting tabletChange =
         new IteratorSetting(1001, "tabletChange", TabletStateChangeIterator.class);
@@ -137,8 +135,10 @@
     try {
       Entry<Key,Value> e = iter.next();
       return createTabletLocationState(e.getKey(), e.getValue());
-    } catch (IOException | BadLocationStateException ex) {
-      throw new RuntimeException(ex);
+    } catch (IOException ex) {
+      throw new UncheckedIOException(ex);
+    } catch (BadLocationStateException ex) {
+      throw new IllegalStateException(ex);
     }
   }
 
@@ -151,8 +151,7 @@
     Location last = null;
     SuspendingTServer suspend = null;
     long lastTimestamp = 0;
-    List<Collection<String>> walogs = new ArrayList<>();
-    boolean chopped = false;
+    List<LogEntry> walogs = new ArrayList<>();
 
     for (Entry<Key,Value> entry : decodedRow.entrySet()) {
 
@@ -176,15 +175,12 @@
         }
         current = location;
       } else if (cf.compareTo(LogColumnFamily.NAME) == 0) {
-        String[] split = entry.getValue().toString().split("\\|")[0].split(";");
-        walogs.add(Arrays.asList(split));
+        walogs.add(LogEntry.fromMetaWalEntry(entry));
       } else if (cf.compareTo(LastLocationColumnFamily.NAME) == 0) {
         if (lastTimestamp < entry.getKey().getTimestamp()) {
           last = Location.last(new TServerInstance(entry.getValue(), cq));
           lastTimestamp = entry.getKey().getTimestamp();
         }
-      } else if (cf.compareTo(ChoppedColumnFamily.NAME) == 0) {
-        chopped = true;
       } else if (TabletColumnFamily.PREV_ROW_COLUMN.equals(cf, cq)) {
         extent = KeyExtent.fromMetaPrevRow(entry);
       } else if (SuspendLocationColumn.SUSPEND_COLUMN.equals(cf, cq)) {
@@ -196,7 +192,7 @@
       log.error(msg);
       throw new BadLocationStateException(msg, k.getRow());
     }
-    return new TabletLocationState(extent, future, current, last, suspend, walogs, chopped);
+    return new TabletLocationState(extent, future, current, last, suspend, walogs);
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/RootTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/RootTabletStateStore.java
index 98123e1..9263b3b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/RootTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/RootTabletStateStore.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.server.manager.state;
 
 import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -27,12 +27,13 @@
 class RootTabletStateStore extends MetaDataStateStore {
 
   RootTabletStateStore(DataLevel level, ClientContext context, CurrentState state) {
-    super(level, context, state, RootTable.NAME);
+    super(level, context, state, AccumuloTable.ROOT.tableName());
   }
 
   @Override
   public ClosableIterator<TabletLocationState> iterator() {
-    return new MetaDataTableScanner(context, TabletsSection.getRange(), state, RootTable.NAME);
+    return new MetaDataTableScanner(context, TabletsSection.getRange(), state,
+        AccumuloTable.ROOT.tableName());
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java
index 42fd9cb..78e3e29 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.server.manager.state;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Base64;
@@ -77,11 +78,12 @@
     merges = parseMerges(options.get(MERGES_OPTION));
     debug = options.containsKey(DEBUG_OPTION);
     migrations = parseMigrations(options.get(MIGRATIONS_OPTION));
+    String managerStateOptionName = options.get(MANAGER_STATE_OPTION);
     try {
-      managerState = ManagerState.valueOf(options.get(MANAGER_STATE_OPTION));
-    } catch (Exception ex) {
-      if (options.get(MANAGER_STATE_OPTION) != null) {
-        log.error("Unable to decode managerState {}", options.get(MANAGER_STATE_OPTION));
+      managerState = ManagerState.valueOf(managerStateOptionName);
+    } catch (RuntimeException ex) {
+      if (managerStateOptionName != null) {
+        log.error("Unable to decode managerState {}", managerStateOptionName);
       }
     }
     Set<TServerInstance> shuttingDown = parseServers(options.get(SHUTTING_DOWN_OPTION));
@@ -103,8 +105,8 @@
         result.add(KeyExtent.readFrom(buffer));
       }
       return result;
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (IOException ex) {
+      throw new UncheckedIOException(ex);
     }
   }
 
@@ -133,7 +135,7 @@
         if (instance != null && instance.endsWith("]")) {
           instance = instance.substring(0, instance.length() - 1);
         }
-        result.add(new TServerInstance(AddressUtil.parseAddress(hostport, false), instance));
+        result.add(new TServerInstance(AddressUtil.parseAddress(hostport), instance));
       }
     }
     return result;
@@ -154,8 +156,8 @@
         result.put(mergeInfo.extent.tableId(), mergeInfo);
       }
       return result;
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (IOException ex) {
+      throw new UncheckedIOException(ex);
     }
   }
 
@@ -256,8 +258,8 @@
           info.write(buffer);
         }
       }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (IOException ex) {
+      throw new UncheckedIOException(ex);
     }
     String encoded =
         Base64.getEncoder().encodeToString(Arrays.copyOf(buffer.getData(), buffer.getLength()));
@@ -270,8 +272,8 @@
       for (KeyExtent extent : migrations) {
         extent.writeTo(buffer);
       }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
+    } catch (IOException ex) {
+      throw new UncheckedIOException(ex);
     }
     String encoded =
         Base64.getEncoder().encodeToString(Arrays.copyOf(buffer.getData(), buffer.getLength()));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateStore.java
index 05072fd..9f4302b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateStore.java
@@ -28,6 +28,7 @@
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.fs.Path;
 
@@ -76,7 +77,7 @@
    * previous tserver.
    */
   void suspend(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      Map<TServerInstance,List<Path>> logsForDeadServers, SteadyTime suspensionTimestamp)
       throws DistributedStoreException;
 
   /**
@@ -91,7 +92,7 @@
   }
 
   public static void suspend(ServerContext context, TabletLocationState tls,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      Map<TServerInstance,List<Path>> logsForDeadServers, SteadyTime suspensionTimestamp)
       throws DistributedStoreException {
     getStoreForTablet(tls.extent, context).suspend(Collections.singletonList(tls),
         logsForDeadServers, suspensionTimestamp);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
index 398e67c..639c72f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/ZooTabletStateStore.java
@@ -20,7 +20,6 @@
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -28,6 +27,7 @@
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
+import org.apache.accumulo.core.metadata.TabletLocationState.BadLocationStateException;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.Ample.ReadConsistency;
@@ -36,6 +36,7 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
@@ -95,16 +96,16 @@
             currentSession = loc;
           }
 
-          List<Collection<String>> logs = new ArrayList<>();
+          List<LogEntry> logs = new ArrayList<>();
           rootMeta.getLogs().forEach(logEntry -> {
-            logs.add(Collections.singleton(logEntry.filename));
-            log.debug("root tablet log {}", logEntry.filename);
+            logs.add(logEntry);
+            log.debug("root tablet log {}", logEntry);
           });
 
           return new TabletLocationState(RootTable.EXTENT, futureSession, currentSession,
-              lastSession, null, logs, false);
-        } catch (Exception ex) {
-          throw new RuntimeException(ex);
+              lastSession, null, logs);
+        } catch (BadLocationStateException ex) {
+          throw new IllegalStateException(ex);
         }
       }
 
@@ -175,8 +176,7 @@
       List<Path> logs = logsForDeadServers.get(futureOrCurrent);
       if (logs != null) {
         for (Path entry : logs) {
-          LogEntry logEntry =
-              new LogEntry(RootTable.EXTENT, System.currentTimeMillis(), entry.toString());
+          LogEntry logEntry = LogEntry.fromPath(entry.toString());
           tabletMutator.putWal(logEntry);
         }
       }
@@ -189,7 +189,7 @@
 
   @Override
   public void suspend(Collection<TabletLocationState> tablets,
-      Map<TServerInstance,List<Path>> logsForDeadServers, long suspensionTimestamp)
+      Map<TServerInstance,List<Path>> logsForDeadServers, SteadyTime suspensionTimestamp)
       throws DistributedStoreException {
     // No support for suspending root tablet.
     unassign(tablets, logsForDeadServers);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
deleted file mode 100644
index 9aea12d..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ /dev/null
@@ -1,374 +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.server.master.balancer;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Set;
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer}
- *             instead, as it as the same functionality but a stable API.
- */
-@Deprecated(since = "2.1.0")
-public class DefaultLoadBalancer extends TabletBalancer {
-
-  private static final Logger log = LoggerFactory.getLogger(DefaultLoadBalancer.class);
-
-  Iterator<TServerInstance> assignments;
-  // if tableToBalance is set, then only balance the given table
-  TableId tableToBalance = null;
-
-  public DefaultLoadBalancer() {
-    log.warn(
-        "{} has been deprecated and will be removed in a future release. Please update your "
-            + "configuration to use the equivalent {} instead.",
-        getClass().getName(), SimpleLoadBalancer.class.getName());
-  }
-
-  public DefaultLoadBalancer(TableId table) {
-    this(); // emit warning
-    tableToBalance = table;
-  }
-
-  List<TServerInstance> randomize(Set<TServerInstance> locations) {
-    List<TServerInstance> result = new ArrayList<>(locations);
-    Collections.shuffle(result);
-    return result;
-  }
-
-  public TServerInstance getAssignment(SortedMap<TServerInstance,TabletServerStatus> locations,
-      TServerInstance last) {
-    if (locations.isEmpty()) {
-      return null;
-    }
-
-    if (last != null) {
-      // Maintain locality
-      String fakeSessionID = " ";
-      TServerInstance simple = new TServerInstance(last.getHostAndPort(), fakeSessionID);
-      Iterator<TServerInstance> find = locations.tailMap(simple).keySet().iterator();
-      if (find.hasNext()) {
-        TServerInstance current = find.next();
-        if (current.getHost().equals(last.getHost())) {
-          return current;
-        }
-      }
-    }
-
-    // The strategy here is to walk through the locations and hand them back, one at a time
-    // Grab an iterator off of the set of options; use a new iterator if it hands back something not
-    // in the current list.
-    if (assignments == null || !assignments.hasNext()) {
-      assignments = randomize(locations.keySet()).iterator();
-    }
-    TServerInstance result = assignments.next();
-    if (!locations.containsKey(result)) {
-      assignments = null;
-      return randomize(locations.keySet()).iterator().next();
-    }
-    return result;
-  }
-
-  static class ServerCounts implements Comparable<ServerCounts> {
-    public final TServerInstance server;
-    public int count;
-    public final TabletServerStatus status;
-
-    ServerCounts(int count, TServerInstance server, TabletServerStatus status) {
-      this.count = count;
-      this.server = server;
-      this.status = status;
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(server) + count;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      return obj == this
-          || (obj != null && obj instanceof ServerCounts && compareTo((ServerCounts) obj) == 0);
-    }
-
-    @Override
-    public int compareTo(ServerCounts obj) {
-      int result = count - obj.count;
-      if (result == 0) {
-        return server.compareTo(obj.server);
-      }
-      return result;
-    }
-  }
-
-  public boolean getMigrations(Map<TServerInstance,TabletServerStatus> current,
-      List<TabletMigration> result) {
-    boolean moreBalancingNeeded = false;
-    try {
-      // no moves possible
-      if (current.size() < 2) {
-        return false;
-      }
-      final Map<TableId,Map<KeyExtent,TabletStats>> donerTabletStats = new HashMap<>();
-
-      // Sort by total number of online tablets, per server
-      int total = 0;
-      ArrayList<ServerCounts> totals = new ArrayList<>();
-      for (Entry<TServerInstance,TabletServerStatus> entry : current.entrySet()) {
-        int serverTotal = 0;
-        if (entry.getValue() != null && entry.getValue().tableMap != null) {
-          for (Entry<String,TableInfo> e : entry.getValue().tableMap.entrySet()) {
-            /**
-             * The check below was on entry.getKey(), but that resolves to a tabletserver not a
-             * tablename. Believe it should be e.getKey() which is a tablename
-             */
-            if (tableToBalance == null || tableToBalance.canonical().equals(e.getKey())) {
-              serverTotal += e.getValue().onlineTablets;
-            }
-          }
-        }
-        totals.add(new ServerCounts(serverTotal, entry.getKey(), entry.getValue()));
-        total += serverTotal;
-      }
-
-      // order from low to high
-      totals.sort(Collections.reverseOrder());
-      int even = total / totals.size();
-      int numServersOverEven = total % totals.size();
-
-      // Move tablets from the servers with too many to the servers with
-      // the fewest but only nominate tablets to move once. This allows us
-      // to fill new servers with tablets from a mostly balanced server
-      // very quickly. However, it may take several balancing passes to move
-      // tablets from one hugely overloaded server to many slightly
-      // under-loaded servers.
-      int end = totals.size() - 1;
-      int movedAlready = 0;
-      int tooManyIndex = 0;
-      while (tooManyIndex < end) {
-        ServerCounts tooMany = totals.get(tooManyIndex);
-        int goal = even;
-        if (tooManyIndex < numServersOverEven) {
-          goal++;
-        }
-        int needToUnload = tooMany.count - goal;
-        ServerCounts tooLittle = totals.get(end);
-        int needToLoad = goal - tooLittle.count - movedAlready;
-        if (needToUnload < 1 && needToLoad < 1) {
-          break;
-        }
-        if (needToUnload >= needToLoad) {
-          result.addAll(move(tooMany, tooLittle, needToLoad, donerTabletStats));
-          end--;
-          movedAlready = 0;
-        } else {
-          result.addAll(move(tooMany, tooLittle, needToUnload, donerTabletStats));
-          movedAlready += needToUnload;
-        }
-        if (needToUnload > needToLoad) {
-          moreBalancingNeeded = true;
-        } else {
-          tooManyIndex++;
-          donerTabletStats.clear();
-        }
-      }
-
-    } finally {
-      log.trace("balance ended with {} migrations", result.size());
-    }
-    return moreBalancingNeeded;
-  }
-
-  /**
-   * Select a tablet based on differences between table loads; if the loads are even, use the
-   * busiest table
-   */
-  List<TabletMigration> move(ServerCounts tooMuch, ServerCounts tooLittle, int count,
-      Map<TableId,Map<KeyExtent,TabletStats>> donerTabletStats) {
-
-    if (count == 0) {
-      return Collections.emptyList();
-    }
-
-    List<TabletMigration> result = new ArrayList<>();
-    // Copy counts so we can update them as we propose migrations
-    Map<TableId,Integer> tooMuchMap = tabletCountsPerTable(tooMuch.status);
-    Map<TableId,Integer> tooLittleMap = tabletCountsPerTable(tooLittle.status);
-
-    for (int i = 0; i < count; i++) {
-      TableId table;
-      Integer tooLittleCount;
-      if (tableToBalance == null) {
-        // find a table to migrate
-        // look for an uneven table count
-        int biggestDifference = 0;
-        TableId biggestDifferenceTable = null;
-        for (var tableEntry : tooMuchMap.entrySet()) {
-          TableId tableID = tableEntry.getKey();
-          tooLittleMap.putIfAbsent(tableID, 0);
-          int diff = tableEntry.getValue() - tooLittleMap.get(tableID);
-          if (diff > biggestDifference) {
-            biggestDifference = diff;
-            biggestDifferenceTable = tableID;
-          }
-        }
-        if (biggestDifference < 2) {
-          table = busiest(tooMuch.status.tableMap);
-        } else {
-          table = biggestDifferenceTable;
-        }
-      } else {
-        // just balance the given table
-        table = tableToBalance;
-      }
-      Map<KeyExtent,TabletStats> onlineTabletsForTable = donerTabletStats.get(table);
-      try {
-        if (onlineTabletsForTable == null) {
-          onlineTabletsForTable = new HashMap<>();
-          List<TabletStats> stats = getOnlineTabletsForTable(tooMuch.server, table);
-          if (stats == null) {
-            log.warn("Unable to find tablets to move");
-            return result;
-          }
-          for (TabletStats stat : stats) {
-            onlineTabletsForTable.put(KeyExtent.fromThrift(stat.extent), stat);
-          }
-          donerTabletStats.put(table, onlineTabletsForTable);
-        }
-      } catch (Exception ex) {
-        log.error("Unable to select a tablet to move", ex);
-        return result;
-      }
-      KeyExtent extent = selectTablet(onlineTabletsForTable);
-      onlineTabletsForTable.remove(extent);
-      if (extent == null) {
-        return result;
-      }
-      tooMuchMap.put(table, tooMuchMap.get(table) - 1);
-      /**
-       * If a table grows from 1 tablet then tooLittleMap.get(table) can return a null, since there
-       * is only one tabletserver that holds all of the tablets. Here we check to see if in fact
-       * that is the case and if so set the value to 0.
-       */
-      tooLittleCount = tooLittleMap.get(table);
-      if (tooLittleCount == null) {
-        tooLittleCount = 0;
-      }
-      tooLittleMap.put(table, tooLittleCount + 1);
-      tooMuch.count--;
-      tooLittle.count++;
-      result.add(new TabletMigration(extent, tooMuch.server, tooLittle.server));
-    }
-    return result;
-  }
-
-  static Map<TableId,Integer> tabletCountsPerTable(TabletServerStatus status) {
-    Map<TableId,Integer> result = new HashMap<>();
-    if (status != null && status.tableMap != null) {
-      Map<String,TableInfo> tableMap = status.tableMap;
-      for (Entry<String,TableInfo> entry : tableMap.entrySet()) {
-        result.put(TableId.of(entry.getKey()), entry.getValue().onlineTablets);
-      }
-    }
-    return result;
-  }
-
-  static KeyExtent selectTablet(Map<KeyExtent,TabletStats> extents) {
-    if (extents.isEmpty()) {
-      return null;
-    }
-    KeyExtent mostRecentlySplit = null;
-    long splitTime = 0;
-    for (Entry<KeyExtent,TabletStats> entry : extents.entrySet()) {
-      if (entry.getValue().splitCreationTime >= splitTime) {
-        splitTime = entry.getValue().splitCreationTime;
-        mostRecentlySplit = entry.getKey();
-      }
-    }
-    return mostRecentlySplit;
-  }
-
-  // define what it means for a tablet to be busy
-  private static TableId busiest(Map<String,TableInfo> tables) {
-    TableId result = null;
-    double busiest = Double.NEGATIVE_INFINITY;
-    for (Entry<String,TableInfo> entry : tables.entrySet()) {
-      TableInfo info = entry.getValue();
-      double busy = info.ingestRate + info.queryRate;
-      if (busy > busiest) {
-        busiest = busy;
-        result = TableId.of(entry.getKey());
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
-      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments) {
-    for (Entry<KeyExtent,TServerInstance> entry : unassigned.entrySet()) {
-      assignments.put(entry.getKey(), getAssignment(current, entry.getValue()));
-    }
-  }
-
-  private static final NoTservers NO_SERVERS = new NoTservers(log);
-
-  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
-
-  @Override
-  public long balance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
-    // do we have any servers?
-    if (current.isEmpty()) {
-      constraintNotMet(NO_SERVERS);
-    } else {
-      // Don't migrate if we have migrations in progress
-      if (migrations.isEmpty()) {
-        resetBalancerErrors();
-        if (getMigrations(current, migrationsOut)) {
-          return 1_000;
-        }
-      } else {
-        outstandingMigrations.migrations = migrations;
-        constraintNotMet(outstandingMigrations);
-      }
-    }
-    return 5_000;
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
deleted file mode 100644
index b251988..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/GroupBalancer.java
+++ /dev/null
@@ -1,751 +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.server.master.balancer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
-import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.function.Function;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.util.ComparablePair;
-import org.apache.accumulo.core.util.MapCounter;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.commons.lang3.mutable.MutableInt;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.HashBasedTable;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Multimap;
-
-/**
- * A balancer that evenly spreads groups of tablets across all tablet server. This balancer
- * accomplishes the following two goals :
- *
- * <ul>
- * <li>Evenly spreads each group across all tservers.
- * <li>Minimizes the total number of groups on each tserver.
- * </ul>
- *
- * <p>
- * To use this balancer you must extend it and implement {@link #getPartitioner()}. See
- * {@link RegexGroupBalancer} as an example.
- *
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.GroupBalancer} instead.
- */
-@Deprecated(since = "2.1.0")
-public abstract class GroupBalancer extends TabletBalancer {
-
-  private final TableId tableId;
-  private long lastRun = 0;
-
-  /**
-   * @return A function that groups tablets into named groups.
-   */
-  protected abstract Function<KeyExtent,String> getPartitioner();
-
-  public GroupBalancer(TableId tableId) {
-    this.tableId = tableId;
-
-    LoggerFactory.getLogger(getClass().getName())
-        .warn("{} has been deprecated and will be "
-            + "removed in a future release. Please update your configuration to use the equivalent "
-            + "{} instead.", getClass().getName(),
-            org.apache.accumulo.core.spi.balancer.GroupBalancer.class.getName());
-
-  }
-
-  protected Map<KeyExtent,TServerInstance> getLocationProvider() {
-    Map<KeyExtent,TServerInstance> tablets = new LinkedHashMap<>();
-    for (var tm : TabletsMetadata.builder(context).forTable(tableId).fetch(LOCATION, PREV_ROW)
-        .build()) {
-      tablets.put(tm.getExtent(), tm.getLocation().getServerInstance());
-    }
-    return tablets;
-  }
-
-  /**
-   * The amount of time to wait between balancing.
-   */
-  protected long getWaitTime() {
-    return 60000;
-  }
-
-  /**
-   * The maximum number of migrations to perform in a single pass.
-   */
-  protected int getMaxMigrations() {
-    return 1000;
-  }
-
-  /**
-   * @return Examine current tserver and migrations and return true if balancing should occur.
-   */
-  protected boolean shouldBalance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations) {
-
-    if (current.size() < 2) {
-      return false;
-    }
-
-    for (KeyExtent keyExtent : migrations) {
-      if (keyExtent.tableId().equals(tableId)) {
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  @Override
-  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
-      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments) {
-
-    if (current.isEmpty()) {
-      return;
-    }
-
-    Function<KeyExtent,String> partitioner = getPartitioner();
-
-    List<ComparablePair<String,KeyExtent>> tabletsByGroup = new ArrayList<>();
-    for (Entry<KeyExtent,TServerInstance> entry : unassigned.entrySet()) {
-      TServerInstance last = entry.getValue();
-      if (last != null) {
-        // Maintain locality
-        String fakeSessionID = " ";
-        TServerInstance simple = new TServerInstance(last.getHostAndPort(), fakeSessionID);
-        Iterator<TServerInstance> find = current.tailMap(simple).keySet().iterator();
-        if (find.hasNext()) {
-          TServerInstance tserver = find.next();
-          if (tserver.getHost().equals(last.getHost())) {
-            assignments.put(entry.getKey(), tserver);
-            continue;
-          }
-        }
-      }
-
-      tabletsByGroup.add(new ComparablePair<>(partitioner.apply(entry.getKey()), entry.getKey()));
-    }
-
-    Collections.sort(tabletsByGroup);
-
-    Iterator<TServerInstance> tserverIter = Iterators.cycle(current.keySet());
-    for (ComparablePair<String,KeyExtent> pair : tabletsByGroup) {
-      KeyExtent ke = pair.getSecond();
-      assignments.put(ke, tserverIter.next());
-    }
-
-  }
-
-  @Override
-  public long balance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
-
-    // The terminology extra and expected are used in this code. Expected tablets is the number of
-    // tablets a tserver must have for a given group and is
-    // numInGroup/numTservers. Extra tablets are any tablets more than the number expected for a
-    // given group. If numInGroup % numTservers > 0, then a tserver
-    // may have one extra tablet for a group.
-    //
-    // Assume we have 4 tservers and group A has 11 tablets.
-    // * expected tablets : group A is expected to have 2 tablets on each tservers
-    // * extra tablets : group A may have an additional tablet on each tserver. Group A has a total
-    // of 3 extra tablets.
-    //
-    // This balancer also evens out the extra tablets across all groups. The terminology
-    // extraExpected and extraExtra is used to describe these tablets.
-    // ExtraExpected is totalExtra/numTservers. ExtraExtra is totalExtra%numTservers. Each tserver
-    // should have at least expectedExtra extra tablets and at most
-    // one extraExtra tablets. All extra tablets on a tserver must be from different groups.
-    //
-    // Assume we have 6 tservers and three groups (G1, G2, G3) with 9 tablets each. Each tserver is
-    // expected to have one tablet from each group and could
-    // possibly have 2 tablets from a group. Below is an illustration of an ideal balancing of extra
-    // tablets. To understand the illustration, the first column
-    // shows tserver T1 with 2 tablets from G1, 1 tablet from G2, and two tablets from G3. EE means
-    // empty, put it there so eclipse formatting would not mess up
-    // table.
-    //
-    // T1 | T2 | T3 | T4 | T5 | T6
-    // ---+----+----+----+----+-----
-    // G3 | G2 | G3 | EE | EE | EE <-- extra extra tablets
-    // G1 | G1 | G1 | G2 | G3 | G2 <-- extra expected tablets.
-    // G1 | G1 | G1 | G1 | G1 | G1 <-- expected tablets for group 1
-    // G2 | G2 | G2 | G2 | G2 | G2 <-- expected tablets for group 2
-    // G3 | G3 | G3 | G3 | G3 | G3 <-- expected tablets for group 3
-    //
-    // Do not want to balance the extra tablets like the following. There are two problem with this.
-    // First extra tablets are not evenly spread. Since there are
-    // a total of 9 extra tablets, every tserver is expected to have at least one extra tablet.
-    // Second tserver T1 has two extra tablet for group G1. This
-    // violates the principal that a tserver can only have one extra tablet for a given group.
-    //
-    // T1 | T2 | T3 | T4 | T5 | T6
-    // ---+----+----+----+----+-----
-    // G1 | EE | EE | EE | EE | EE <--- one extra tablets from group 1
-    // G3 | G3 | G3 | EE | EE | EE <--- three extra tablets from group 3
-    // G2 | G2 | G2 | EE | EE | EE <--- three extra tablets from group 2
-    // G1 | G1 | EE | EE | EE | EE <--- two extra tablets from group 1
-    // G1 | G1 | G1 | G1 | G1 | G1 <-- expected tablets for group 1
-    // G2 | G2 | G2 | G2 | G2 | G2 <-- expected tablets for group 2
-    // G3 | G3 | G3 | G3 | G3 | G3 <-- expected tablets for group 3
-
-    if (!shouldBalance(current, migrations)) {
-      return 5000;
-    }
-
-    if (System.currentTimeMillis() - lastRun < getWaitTime()) {
-      return 5000;
-    }
-
-    MapCounter<String> groupCounts = new MapCounter<>();
-    Map<TServerInstance,TserverGroupInfo> tservers = new HashMap<>();
-
-    for (TServerInstance tsi : current.keySet()) {
-      tservers.put(tsi, new TserverGroupInfo(tsi));
-    }
-
-    Function<KeyExtent,String> partitioner = getPartitioner();
-
-    // collect stats about current state
-    for (var tablet : getLocationProvider().entrySet()) {
-      String group = partitioner.apply(tablet.getKey());
-      var loc = tablet.getValue();
-
-      if (loc == null || !tservers.containsKey(loc)) {
-        return 5000;
-      }
-
-      groupCounts.increment(group, 1);
-      TserverGroupInfo tgi = tservers.get(loc);
-      tgi.addGroup(group);
-    }
-
-    Map<String,Integer> expectedCounts = new HashMap<>();
-
-    int totalExtra = 0;
-    for (String group : groupCounts.keySet()) {
-      int groupCount = groupCounts.getInt(group);
-      totalExtra += groupCount % current.size();
-      expectedCounts.put(group, (groupCount / current.size()));
-    }
-
-    // The number of extra tablets from all groups that each tserver must have.
-    int expectedExtra = totalExtra / current.size();
-    int maxExtraGroups = expectedExtra + 1;
-
-    expectedCounts = Collections.unmodifiableMap(expectedCounts);
-    tservers = Collections.unmodifiableMap(tservers);
-
-    for (TserverGroupInfo tgi : tservers.values()) {
-      tgi.finishedAdding(expectedCounts);
-    }
-
-    Moves moves = new Moves();
-
-    // The order of the following steps is important, because as ordered each step should not move
-    // any tablets moved by a previous step.
-    balanceExpected(tservers, moves);
-    if (moves.size() < getMaxMigrations()) {
-      balanceExtraExpected(tservers, expectedExtra, moves);
-      if (moves.size() < getMaxMigrations()) {
-        boolean cont = balanceExtraMultiple(tservers, maxExtraGroups, moves);
-        if (cont && moves.size() < getMaxMigrations()) {
-          balanceExtraExtra(tservers, maxExtraGroups, moves);
-        }
-      }
-    }
-
-    populateMigrations(tservers.keySet(), migrationsOut, moves);
-
-    lastRun = System.currentTimeMillis();
-
-    return 5000;
-  }
-
-  static class TserverGroupInfo {
-
-    private Map<String,Integer> expectedCounts;
-    private final Map<String,MutableInt> initialCounts = new HashMap<>();
-    private final Map<String,Integer> extraCounts = new HashMap<>();
-    private final Map<String,Integer> expectedDeficits = new HashMap<>();
-
-    private final TServerInstance tsi;
-    private boolean finishedAdding = false;
-
-    TserverGroupInfo(TServerInstance tsi) {
-      this.tsi = tsi;
-    }
-
-    public void addGroup(String group) {
-      checkState(!finishedAdding);
-
-      MutableInt mi = initialCounts.get(group);
-      if (mi == null) {
-        mi = new MutableInt();
-        initialCounts.put(group, mi);
-      }
-
-      mi.increment();
-    }
-
-    public void finishedAdding(Map<String,Integer> expectedCounts) {
-      checkState(!finishedAdding);
-      finishedAdding = true;
-      this.expectedCounts = expectedCounts;
-
-      for (Entry<String,Integer> entry : expectedCounts.entrySet()) {
-        String group = entry.getKey();
-        int expected = entry.getValue();
-
-        MutableInt count = initialCounts.get(group);
-        int num = count == null ? 0 : count.intValue();
-
-        if (num < expected) {
-          expectedDeficits.put(group, expected - num);
-        } else if (num > expected) {
-          extraCounts.put(group, num - expected);
-        }
-      }
-
-    }
-
-    public void moveOff(String group, int num) {
-      checkArgument(num > 0);
-      checkState(finishedAdding);
-
-      Integer extraCount = extraCounts.get(group);
-
-      // don't wrap precondition check due to https://github.com/spotbugs/spotbugs/issues/462
-      String formatString = "group=%s num=%s extraCount=%s";
-      checkArgument(extraCount != null && extraCount >= num, formatString, group, num, extraCount);
-
-      MutableInt initialCount = initialCounts.get(group);
-
-      checkArgument(initialCount.intValue() >= num);
-
-      initialCount.subtract(num);
-
-      if (extraCount - num == 0) {
-        extraCounts.remove(group);
-      } else {
-        extraCounts.put(group, extraCount - num);
-      }
-    }
-
-    public void moveTo(String group, int num) {
-      checkArgument(num > 0);
-      checkArgument(expectedCounts.containsKey(group));
-      checkState(finishedAdding);
-
-      Integer deficit = expectedDeficits.get(group);
-      if (deficit != null) {
-        if (num >= deficit) {
-          expectedDeficits.remove(group);
-          num -= deficit;
-        } else {
-          expectedDeficits.put(group, deficit - num);
-          num = 0;
-        }
-      }
-
-      if (num > 0) {
-        Integer extra = extraCounts.get(group);
-        if (extra == null) {
-          extra = 0;
-        }
-
-        extraCounts.put(group, extra + num);
-      }
-
-      // TODO could check extra constraints
-    }
-
-    public Map<String,Integer> getExpectedDeficits() {
-      checkState(finishedAdding);
-      return Collections.unmodifiableMap(expectedDeficits);
-    }
-
-    public Map<String,Integer> getExtras() {
-      checkState(finishedAdding);
-      return Collections.unmodifiableMap(extraCounts);
-    }
-
-    public TServerInstance getTserverInstance() {
-      return tsi;
-    }
-
-    @Override
-    public int hashCode() {
-      return tsi.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof TserverGroupInfo) {
-        TserverGroupInfo otgi = (TserverGroupInfo) o;
-        return tsi.equals(otgi.tsi);
-      }
-
-      return false;
-    }
-
-    @Override
-    public String toString() {
-      return tsi.getHostPortSession();
-    }
-
-  }
-
-  private static class Move {
-    TserverGroupInfo dest;
-    int count;
-
-    public Move(TserverGroupInfo dest, int num) {
-      this.dest = dest;
-      this.count = num;
-    }
-  }
-
-  private static class Moves {
-
-    private final HashBasedTable<TServerInstance,String,List<Move>> moves = HashBasedTable.create();
-    private int totalMoves = 0;
-
-    public void move(String group, int num, TserverGroupInfo src, TserverGroupInfo dest) {
-      checkArgument(num > 0);
-      checkArgument(!src.equals(dest));
-
-      src.moveOff(group, num);
-      dest.moveTo(group, num);
-
-      List<Move> srcMoves = moves.get(src.getTserverInstance(), group);
-      if (srcMoves == null) {
-        srcMoves = new ArrayList<>();
-        moves.put(src.getTserverInstance(), group, srcMoves);
-      }
-
-      srcMoves.add(new Move(dest, num));
-      totalMoves += num;
-    }
-
-    public TServerInstance removeMove(TServerInstance src, String group) {
-      List<Move> srcMoves = moves.get(src, group);
-      if (srcMoves == null) {
-        return null;
-      }
-
-      Move move = srcMoves.get(srcMoves.size() - 1);
-      TServerInstance ret = move.dest.getTserverInstance();
-      totalMoves--;
-
-      move.count--;
-      if (move.count == 0) {
-        srcMoves.remove(srcMoves.size() - 1);
-        if (srcMoves.isEmpty()) {
-          moves.remove(src, group);
-        }
-      }
-
-      return ret;
-    }
-
-    public int size() {
-      return totalMoves;
-    }
-  }
-
-  private void balanceExtraExtra(Map<TServerInstance,TserverGroupInfo> tservers, int maxExtraGroups,
-      Moves moves) {
-    HashBasedTable<String,TServerInstance,TserverGroupInfo> surplusExtra = HashBasedTable.create();
-    for (TserverGroupInfo tgi : tservers.values()) {
-      Map<String,Integer> extras = tgi.getExtras();
-      if (extras.size() > maxExtraGroups) {
-        for (String group : extras.keySet()) {
-          surplusExtra.put(group, tgi.getTserverInstance(), tgi);
-        }
-      }
-    }
-
-    ArrayList<Pair<String,TServerInstance>> serversGroupsToRemove = new ArrayList<>();
-    ArrayList<TServerInstance> serversToRemove = new ArrayList<>();
-
-    for (TserverGroupInfo destTgi : tservers.values()) {
-      if (surplusExtra.isEmpty()) {
-        break;
-      }
-
-      Map<String,Integer> extras = destTgi.getExtras();
-      if (extras.size() < maxExtraGroups) {
-        serversToRemove.clear();
-        serversGroupsToRemove.clear();
-        for (String group : surplusExtra.rowKeySet()) {
-          if (!extras.containsKey(group)) {
-            TserverGroupInfo srcTgi = surplusExtra.row(group).values().iterator().next();
-
-            moves.move(group, 1, srcTgi, destTgi);
-
-            if (srcTgi.getExtras().size() <= maxExtraGroups) {
-              serversToRemove.add(srcTgi.getTserverInstance());
-            } else {
-              serversGroupsToRemove.add(new Pair<>(group, srcTgi.getTserverInstance()));
-            }
-
-            if (destTgi.getExtras().size() >= maxExtraGroups
-                || moves.size() >= getMaxMigrations()) {
-              break;
-            }
-          }
-        }
-
-        if (!serversToRemove.isEmpty()) {
-          surplusExtra.columnKeySet().removeAll(serversToRemove);
-        }
-
-        for (Pair<String,TServerInstance> pair : serversGroupsToRemove) {
-          surplusExtra.remove(pair.getFirst(), pair.getSecond());
-        }
-
-        if (moves.size() >= getMaxMigrations()) {
-          break;
-        }
-      }
-    }
-  }
-
-  private boolean balanceExtraMultiple(Map<TServerInstance,TserverGroupInfo> tservers,
-      int maxExtraGroups, Moves moves) {
-    Multimap<String,TserverGroupInfo> extraMultiple = HashMultimap.create();
-
-    for (TserverGroupInfo tgi : tservers.values()) {
-      Map<String,Integer> extras = tgi.getExtras();
-      for (Entry<String,Integer> entry : extras.entrySet()) {
-        if (entry.getValue() > 1) {
-          extraMultiple.put(entry.getKey(), tgi);
-        }
-      }
-    }
-
-    balanceExtraMultiple(tservers, maxExtraGroups, moves, extraMultiple, false);
-    if (moves.size() < getMaxMigrations() && !extraMultiple.isEmpty()) {
-      // no place to move so must exceed maxExtra temporarily... subsequent balancer calls will
-      // smooth things out
-      balanceExtraMultiple(tservers, maxExtraGroups, moves, extraMultiple, true);
-      return false;
-    } else {
-      return true;
-    }
-  }
-
-  private void balanceExtraMultiple(Map<TServerInstance,TserverGroupInfo> tservers,
-      int maxExtraGroups, Moves moves, Multimap<String,TserverGroupInfo> extraMultiple,
-      boolean alwaysAdd) {
-
-    ArrayList<Pair<String,TserverGroupInfo>> serversToRemove = new ArrayList<>();
-    for (TserverGroupInfo destTgi : tservers.values()) {
-      Map<String,Integer> extras = destTgi.getExtras();
-      if (alwaysAdd || extras.size() < maxExtraGroups) {
-        serversToRemove.clear();
-        for (String group : extraMultiple.keySet()) {
-          if (!extras.containsKey(group)) {
-            Collection<TserverGroupInfo> sources = extraMultiple.get(group);
-            Iterator<TserverGroupInfo> iter = sources.iterator();
-            TserverGroupInfo srcTgi = iter.next();
-
-            int num = srcTgi.getExtras().get(group);
-
-            moves.move(group, 1, srcTgi, destTgi);
-
-            if (num == 2) {
-              serversToRemove.add(new Pair<>(group, srcTgi));
-            }
-
-            if (destTgi.getExtras().size() >= maxExtraGroups
-                || moves.size() >= getMaxMigrations()) {
-              break;
-            }
-          }
-        }
-
-        for (Pair<String,TserverGroupInfo> pair : serversToRemove) {
-          extraMultiple.remove(pair.getFirst(), pair.getSecond());
-        }
-
-        if (extraMultiple.isEmpty() || moves.size() >= getMaxMigrations()) {
-          break;
-        }
-      }
-    }
-  }
-
-  private void balanceExtraExpected(Map<TServerInstance,TserverGroupInfo> tservers,
-      int expectedExtra, Moves moves) {
-
-    HashBasedTable<String,TServerInstance,TserverGroupInfo> extraSurplus = HashBasedTable.create();
-
-    for (TserverGroupInfo tgi : tservers.values()) {
-      Map<String,Integer> extras = tgi.getExtras();
-      if (extras.size() > expectedExtra) {
-        for (String group : extras.keySet()) {
-          extraSurplus.put(group, tgi.getTserverInstance(), tgi);
-        }
-      }
-    }
-
-    ArrayList<TServerInstance> emptyServers = new ArrayList<>();
-    ArrayList<Pair<String,TServerInstance>> emptyServerGroups = new ArrayList<>();
-    for (TserverGroupInfo destTgi : tservers.values()) {
-      if (extraSurplus.isEmpty()) {
-        break;
-      }
-
-      Map<String,Integer> extras = destTgi.getExtras();
-      if (extras.size() < expectedExtra) {
-        emptyServers.clear();
-        emptyServerGroups.clear();
-        nextGroup: for (String group : extraSurplus.rowKeySet()) {
-          if (!extras.containsKey(group)) {
-            Iterator<TserverGroupInfo> iter = extraSurplus.row(group).values().iterator();
-            TserverGroupInfo srcTgi = iter.next();
-
-            while (srcTgi.getExtras().size() <= expectedExtra) {
-              if (iter.hasNext()) {
-                srcTgi = iter.next();
-              } else {
-                continue nextGroup;
-              }
-            }
-
-            moves.move(group, 1, srcTgi, destTgi);
-
-            if (srcTgi.getExtras().size() <= expectedExtra) {
-              emptyServers.add(srcTgi.getTserverInstance());
-            } else if (srcTgi.getExtras().get(group) == null) {
-              emptyServerGroups.add(new Pair<>(group, srcTgi.getTserverInstance()));
-            }
-
-            if (destTgi.getExtras().size() >= expectedExtra || moves.size() >= getMaxMigrations()) {
-              break;
-            }
-          }
-        }
-
-        if (!emptyServers.isEmpty()) {
-          extraSurplus.columnKeySet().removeAll(emptyServers);
-        }
-
-        for (Pair<String,TServerInstance> pair : emptyServerGroups) {
-          extraSurplus.remove(pair.getFirst(), pair.getSecond());
-        }
-
-        if (moves.size() >= getMaxMigrations()) {
-          break;
-        }
-      }
-    }
-  }
-
-  private void balanceExpected(Map<TServerInstance,TserverGroupInfo> tservers, Moves moves) {
-    Multimap<String,TserverGroupInfo> groupDefecits = HashMultimap.create();
-    Multimap<String,TserverGroupInfo> groupSurplus = HashMultimap.create();
-
-    for (TserverGroupInfo tgi : tservers.values()) {
-      for (String group : tgi.getExpectedDeficits().keySet()) {
-        groupDefecits.put(group, tgi);
-      }
-
-      for (String group : tgi.getExtras().keySet()) {
-        groupSurplus.put(group, tgi);
-      }
-    }
-
-    for (String group : groupDefecits.keySet()) {
-      Collection<TserverGroupInfo> defecitServers = groupDefecits.get(group);
-      for (TserverGroupInfo defecitTsi : defecitServers) {
-        int numToMove = defecitTsi.getExpectedDeficits().get(group);
-
-        Iterator<TserverGroupInfo> surplusIter = groupSurplus.get(group).iterator();
-        while (numToMove > 0) {
-          TserverGroupInfo surplusTsi = surplusIter.next();
-
-          int available = surplusTsi.getExtras().get(group);
-
-          if (numToMove >= available) {
-            surplusIter.remove();
-          }
-
-          int transfer = Math.min(numToMove, available);
-
-          numToMove -= transfer;
-
-          moves.move(group, transfer, surplusTsi, defecitTsi);
-          if (moves.size() >= getMaxMigrations()) {
-            return;
-          }
-        }
-      }
-    }
-  }
-
-  private void populateMigrations(Set<TServerInstance> current, List<TabletMigration> migrationsOut,
-      Moves moves) {
-    if (moves.size() == 0) {
-      return;
-    }
-
-    Function<KeyExtent,String> partitioner = getPartitioner();
-
-    for (var tablet : getLocationProvider().entrySet()) {
-      String group = partitioner.apply(tablet.getKey());
-      var loc = tablet.getValue();
-
-      if (loc == null || !current.contains(loc)) {
-        migrationsOut.clear();
-        return;
-      }
-
-      TServerInstance dest = moves.removeMove(loc, group);
-      if (dest != null) {
-        migrationsOut.add(new TabletMigration(tablet.getKey(), loc, dest));
-        if (moves.size() == 0) {
-          break;
-        }
-      }
-    }
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
deleted file mode 100644
index a965b5f..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
+++ /dev/null
@@ -1,576 +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.server.master.balancer;
-
-import static java.util.concurrent.TimeUnit.HOURS;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.AccumuloConfiguration.Deriver;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-import org.apache.thrift.TException;
-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.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-
-/**
- * This balancer creates groups of tablet servers using user-provided regular expressions over the
- * tablet server hostnames. Then it delegates to the table balancer to balance the tablets within
- * the resulting group of tablet servers. All tablet servers that do not match a regex are grouped
- * into a default group.<br>
- * Regex properties for this balancer are specified as:<br>
- * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
- * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets
- * that it should not be according to the regex configuration. If this occurs then the offending
- * tablets will be reassigned. This would cover the case where the configuration is changed and the
- * manager is restarted while the tablet servers are up. To change the out of bounds check time
- * period, set the following property:<br>
- * <b>table.custom.balancer.host.regex.oob.period=5m</b><br>
- * Regex matching can be based on either the host name (default) or host ip address. To set this
- * balancer to match the regular expressions to the tablet server IP address, then set the following
- * property:<br>
- * <b>table.custom.balancer.host.regex.is.ip=true</b><br>
- * It's possible that this balancer may create a lot of migrations. To limit the number of
- * migrations that are created during a balance call, set the following property (default 250):<br>
- * <b>table.custom.balancer.host.regex.concurrent.migrations</b> This balancer can continue
- * balancing even if there are outstanding migrations. To limit the number of outstanding migrations
- * in which this balancer will continue balancing, set the following property (default 0):<br>
- * <b>table.custom.balancer.host.regex.max.outstanding.migrations</b>
- *
- * @deprecated since 2.1.0. Use
- *             {@link org.apache.accumulo.core.spi.balancer.HostRegexTableLoadBalancer} instead
- */
-@Deprecated(since = "2.1.0")
-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);
-  public static final String HOST_BALANCER_PREFIX = PROP_PREFIX + "balancer.host.regex.";
-  public static final String HOST_BALANCER_OOB_CHECK_KEY =
-      PROP_PREFIX + "balancer.host.regex.oob.period";
-  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
-  public static final String HOST_BALANCER_REGEX_USING_IPS_KEY =
-      PROP_PREFIX + "balancer.host.regex.is.ip";
-  public static final String HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY =
-      PROP_PREFIX + "balancer.host.regex.concurrent.migrations";
-  private static final int HOST_BALANCER_REGEX_MAX_MIGRATIONS_DEFAULT = 250;
-  protected static final String DEFAULT_POOL = "HostTableLoadBalancer.ALL";
-  private static final int DEFAULT_OUTSTANDING_MIGRATIONS = 0;
-  public static final String HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY =
-      PROP_PREFIX + "balancer.host.regex.max.outstanding.migrations";
-
-  private static Map<String,String> getRegexes(AccumuloConfiguration aconf) {
-    Map<String,String> regexes = new HashMap<>();
-    Map<String,String> customProps =
-        aconf.getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
-
-    if (customProps != null && !customProps.isEmpty()) {
-      for (Entry<String,String> customProp : customProps.entrySet()) {
-        if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
-          if (customProp.getKey().equals(HOST_BALANCER_OOB_CHECK_KEY)
-              || customProp.getKey().equals(HOST_BALANCER_REGEX_USING_IPS_KEY)
-              || customProp.getKey().equals(HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY)
-              || customProp.getKey().equals(HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY)) {
-            continue;
-          }
-          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
-          String regex = customProp.getValue();
-          regexes.put(tableName, regex);
-        }
-      }
-    }
-
-    return Map.copyOf(regexes);
-  }
-
-  /**
-   * Host Regex Table Load Balance Config
-   */
-  static class HrtlbConf {
-
-    protected long oobCheckMillis =
-        ConfigurationTypeHelper.getTimeInMillis(HOST_BALANCER_OOB_DEFAULT);
-    private int maxTServerMigrations = HOST_BALANCER_REGEX_MAX_MIGRATIONS_DEFAULT;
-    private int maxOutstandingMigrations = DEFAULT_OUTSTANDING_MIGRATIONS;
-    private boolean isIpBasedRegex = false;
-    private Map<String,String> regexes;
-    private Map<String,Pattern> poolNameToRegexPattern = null;
-
-    HrtlbConf(AccumuloConfiguration aconf) {
-      System.out.println("building hrtlb conf");
-      String oobProperty = aconf.get(HOST_BALANCER_OOB_CHECK_KEY);
-      if (oobProperty != null) {
-        oobCheckMillis = ConfigurationTypeHelper.getTimeInMillis(oobProperty);
-      }
-      String ipBased = aconf.get(HOST_BALANCER_REGEX_USING_IPS_KEY);
-      if (ipBased != null) {
-        isIpBasedRegex = Boolean.parseBoolean(ipBased);
-      }
-      String migrations = aconf.get(HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY);
-      if (migrations != null) {
-        maxTServerMigrations = Integer.parseInt(migrations);
-      }
-      String outstanding = aconf.get(HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY);
-      if (outstanding != null) {
-        maxOutstandingMigrations = Integer.parseInt(outstanding);
-      }
-
-      this.regexes = getRegexes(aconf);
-
-      Map<String,Pattern> poolNameToRegexPatternBuilder = new HashMap<>();
-      regexes.forEach((k, v) -> {
-        poolNameToRegexPatternBuilder.put(k, Pattern.compile(v));
-      });
-
-      poolNameToRegexPattern = Map.copyOf(poolNameToRegexPatternBuilder);
-    }
-  }
-
-  private static final Set<KeyExtent> EMPTY_MIGRATIONS = Collections.emptySet();
-  private volatile long lastOOBCheck = System.currentTimeMillis();
-  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<>();
-  private final Map<KeyExtent,TabletMigration> migrationsFromLastPass = new HashMap<>();
-  private final Map<String,Long> tableToTimeSinceNoMigrations = new HashMap<>();
-
-  private Deriver<HrtlbConf> hrtlbConf;
-  private LoadingCache<TableId,Deriver<Map<String,String>>> tablesRegExCache;
-
-  /**
-   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into
-   * a default pool. This could be expensive in the terms of the amount of time to recompute the
-   * groups, so HOST_BALANCER_POOL_RECHECK_KEY should be specified in the terms of minutes, not
-   * seconds or less.
-   *
-   * @param current map of current tservers
-   * @return current servers grouped by pool name, if not a match it is put into a default pool.
-   */
-  protected synchronized Map<String,SortedMap<TServerInstance,TabletServerStatus>>
-      splitCurrentByRegex(SortedMap<TServerInstance,TabletServerStatus> current) {
-    LOG.debug("Performing pool recheck - regrouping tablet servers based on regular expressions");
-    Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<>();
-    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-      List<String> poolNames = getPoolNamesForHost(e.getKey().getHost());
-      for (String pool : poolNames) {
-        SortedMap<TServerInstance,TabletServerStatus> np = newPools.get(pool);
-        if (np == null) {
-          np = new TreeMap<>(current.comparator());
-          newPools.put(pool, np);
-        }
-        np.put(e.getKey(), e.getValue());
-      }
-    }
-
-    if (newPools.get(DEFAULT_POOL) == null) {
-      LOG.warn("Default pool is empty; assigning all tablet servers to the default pool");
-      SortedMap<TServerInstance,TabletServerStatus> dp = new TreeMap<>(current.comparator());
-      dp.putAll(current);
-      newPools.put(DEFAULT_POOL, dp);
-    }
-
-    pools = newPools;
-
-    LOG.trace("Pool to TabletServer mapping:");
-    if (LOG.isTraceEnabled()) {
-      for (Entry<String,SortedMap<TServerInstance,TabletServerStatus>> e : pools.entrySet()) {
-        LOG.trace("\tpool: {} -> tservers: {}", e.getKey(), e.getValue().keySet());
-      }
-    }
-    return pools;
-  }
-
-  /**
-   * Matches host against the regexes and returns the matching pool names
-   *
-   * @param host tablet server host
-   * @return pool names, will return default pool if host matches more no regex
-   */
-  protected List<String> getPoolNamesForHost(String host) {
-    String test = host;
-    if (!hrtlbConf.derive().isIpBasedRegex) {
-      try {
-        test = getNameFromIp(host);
-      } catch (UnknownHostException e1) {
-        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool",
-            e1);
-        return Collections.singletonList(DEFAULT_POOL);
-      }
-    }
-    List<String> pools = new ArrayList<>();
-    for (Entry<String,Pattern> e : hrtlbConf.derive().poolNameToRegexPattern.entrySet()) {
-      if (e.getValue().matcher(test).matches()) {
-        pools.add(e.getKey());
-      }
-    }
-    if (pools.isEmpty()) {
-      pools.add(DEFAULT_POOL);
-    }
-    return pools;
-  }
-
-  protected String getNameFromIp(String hostIp) throws UnknownHostException {
-    return InetAddress.getByName(hostIp).getHostName();
-  }
-
-  private void checkTableConfig(TableId tableId) {
-    Map<String,String> tableRegexes = tablesRegExCache.getUnchecked(tableId).derive();
-
-    if (!hrtlbConf.derive().regexes.equals(tableRegexes)) {
-      LoggerFactory.getLogger(HostRegexTableLoadBalancer.class).warn(
-          "Table id {} has different config than system.  The per table config is ignored.",
-          tableId);
-    }
-  }
-
-  private Map<TableId,String> createdTableNameMap(Map<String,String> tableIdMap) {
-    HashMap<TableId,String> tableNameMap = new HashMap<>();
-    tableIdMap.forEach((tableName, tableId) -> {
-      tableNameMap.put(TableId.of(tableId), tableName);
-    });
-    return tableNameMap;
-  }
-
-  /**
-   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
-   *
-   * @param tableName name of table
-   * @return tablet server pool name (table name or DEFAULT_POOL)
-   */
-  protected String getPoolNameForTable(String tableName) {
-    if (tableName == null) {
-      return DEFAULT_POOL;
-    }
-    return hrtlbConf.derive().poolNameToRegexPattern.containsKey(tableName) ? tableName
-        : DEFAULT_POOL;
-  }
-
-  @Override
-  public String toString() {
-    HrtlbConf myConf = hrtlbConf.derive();
-    ToStringBuilder buf = new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    buf.append("\nTablet Out Of Bounds Check Interval", myConf.oobCheckMillis);
-    buf.append("\nMax Tablet Server Migrations", myConf.maxTServerMigrations);
-    buf.append("\nRegular Expressions use IPs", myConf.isIpBasedRegex);
-    buf.append("\nPools", myConf.poolNameToRegexPattern);
-    return buf.toString();
-  }
-
-  public Map<String,Pattern> getPoolNameToRegexPattern() {
-    return hrtlbConf.derive().poolNameToRegexPattern;
-  }
-
-  public int getMaxMigrations() {
-    return hrtlbConf.derive().maxTServerMigrations;
-  }
-
-  public int getMaxOutstandingMigrations() {
-    return hrtlbConf.derive().maxOutstandingMigrations;
-  }
-
-  public long getOobCheckMillis() {
-    return hrtlbConf.derive().oobCheckMillis;
-  }
-
-  public boolean isIpBasedRegex() {
-    return hrtlbConf.derive().isIpBasedRegex;
-  }
-
-  @Override
-  public void init(ServerContext context) {
-    super.init(context);
-
-    this.hrtlbConf = context.getConfiguration().newDeriver(HrtlbConf::new);
-
-    tablesRegExCache =
-        CacheBuilder.newBuilder().expireAfterAccess(1, HOURS).build(new CacheLoader<>() {
-          @Override
-          public Deriver<Map<String,String>> load(TableId key) throws Exception {
-            return context.getTableConfiguration(key)
-                .newDeriver(HostRegexTableLoadBalancer::getRegexes);
-          }
-        });
-
-    LOG.info("{}", this);
-  }
-
-  @Override
-  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
-      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments) {
-
-    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
-    // group the unassigned into tables
-    Map<TableId,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<>();
-    unassigned.forEach((ke, lastTserver) -> groupedUnassigned
-        .computeIfAbsent(ke.tableId(), k -> new HashMap<>()).put(ke, lastTserver));
-
-    Map<TableId,String> tableIdToTableName = createdTableNameMap(getTableOperations().tableIdMap());
-
-    // Send a view of the current servers to the tables tablet balancer
-    for (Entry<TableId,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
-      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<>();
-      String tableName = tableIdToTableName.get(e.getKey());
-      String poolName = getPoolNameForTable(tableName);
-      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
-      if (currentView == null || currentView.isEmpty()) {
-        LOG.warn("No tablet servers online for table {}, assigning within default pool", tableName);
-        currentView = pools.get(DEFAULT_POOL);
-        if (currentView == null) {
-          LOG.error(
-              "No tablet servers exist in the default pool, unable to assign tablets for table {}",
-              tableName);
-          continue;
-        }
-      }
-      LOG.debug("Sending {} tablets to balancer for table {} for assignment within tservers {}",
-          e.getValue().size(), tableName, currentView.keySet());
-      getBalancerForTable(e.getKey()).getAssignments(currentView, e.getValue(), newAssignments);
-      assignments.putAll(newAssignments);
-    }
-  }
-
-  @Override
-  public long balance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
-    long minBalanceTime = 20_000;
-    // Iterate over the tables and balance each of them
-    TableOperations t = getTableOperations();
-    if (t == null) {
-      return minBalanceTime;
-    }
-
-    Map<String,String> tableIdMap = t.tableIdMap();
-    Map<TableId,String> tableIdToTableName = createdTableNameMap(tableIdMap);
-    tableIdToTableName.keySet().forEach(this::checkTableConfig);
-
-    long now = System.currentTimeMillis();
-
-    HrtlbConf myConf = hrtlbConf.derive();
-
-    Map<String,SortedMap<TServerInstance,TabletServerStatus>> currentGrouped =
-        splitCurrentByRegex(current);
-    if ((now - this.lastOOBCheck) > myConf.oobCheckMillis) {
-      try {
-        // Check to see if a tablet is assigned outside the bounds of the pool. If so, migrate it.
-        for (String table : tableIdMap.keySet()) {
-          LOG.debug("Checking for out of bounds tablets for table {}", table);
-          String tablePoolName = getPoolNameForTable(table);
-          for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-            // pool names are the same as table names, except in the DEFAULT case.
-            // If this table is assigned to a pool for this host, then move on.
-            List<String> hostPools = getPoolNamesForHost(e.getKey().getHost());
-            if (hostPools.contains(tablePoolName)) {
-              continue;
-            }
-            String tid = tableIdMap.get(table);
-            if (tid == null) {
-              LOG.warn("Unable to check for out of bounds tablets for table {},"
-                  + " it may have been deleted or renamed.", table);
-              continue;
-            }
-            try {
-              List<TabletStats> outOfBoundsTablets =
-                  getOnlineTabletsForTable(e.getKey(), TableId.of(tid));
-              if (outOfBoundsTablets == null) {
-                continue;
-              }
-              for (TabletStats ts : outOfBoundsTablets) {
-                KeyExtent ke = KeyExtent.fromThrift(ts.getExtent());
-                if (migrations.contains(ke)) {
-                  LOG.debug("Migration for out of bounds tablet {} has already been requested", ke);
-                  continue;
-                }
-                String poolName = getPoolNameForTable(table);
-                SortedMap<TServerInstance,TabletServerStatus> currentView =
-                    currentGrouped.get(poolName);
-                if (currentView != null) {
-                  int skip = random.nextInt(currentView.size());
-                  Iterator<TServerInstance> iter = currentView.keySet().iterator();
-                  for (int i = 0; i < skip; i++) {
-                    iter.next();
-                  }
-                  TServerInstance nextTS = iter.next();
-                  LOG.info("Tablet {} is currently outside the bounds of the"
-                      + " regex, migrating from {} to {}", ke, e.getKey(), nextTS);
-                  migrationsOut.add(new TabletMigration(ke, e.getKey(), nextTS));
-                  if (migrationsOut.size() >= myConf.maxTServerMigrations) {
-                    break;
-                  }
-                } else {
-                  LOG.warn("No tablet servers online for pool {}, unable to"
-                      + " migrate out of bounds tablets", poolName);
-                }
-              }
-            } catch (TException e1) {
-              LOG.error("Error in OOB check getting tablets for table {} from server {} {}", tid,
-                  e.getKey().getHost(), e);
-            }
-          }
-        }
-      } finally {
-        // this could have taken a while...get a new time
-        this.lastOOBCheck = System.currentTimeMillis();
-      }
-    }
-
-    if (!migrationsOut.isEmpty()) {
-      LOG.warn("Not balancing tables due to moving {} out of bounds tablets", migrationsOut.size());
-      LOG.info("Migrating out of bounds tablets: {}", migrationsOut);
-      return minBalanceTime;
-    }
-
-    if (migrations != null && !migrations.isEmpty()) {
-      if (migrations.size() >= myConf.maxOutstandingMigrations) {
-        LOG.warn("Not balancing tables due to {} outstanding migrations", migrations.size());
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Sample up to 10 outstanding migrations: {}", limitTen(migrations));
-        }
-        return minBalanceTime;
-      }
-
-      LOG.debug("Current outstanding migrations of {} being applied", migrations.size());
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Sample up to 10 outstanding migrations: {}", limitTen(migrations));
-      }
-      migrationsFromLastPass.keySet().retainAll(migrations);
-      SortedMap<TServerInstance,TabletServerStatus> currentCopy = new TreeMap<>(current);
-      Multimap<TServerInstance,String> serverTableIdCopied = HashMultimap.create();
-      for (TabletMigration migration : migrationsFromLastPass.values()) {
-        TableInfo fromInfo = getTableInfo(currentCopy, serverTableIdCopied,
-            migration.tablet.tableId().toString(), migration.oldServer);
-        if (fromInfo != null) {
-          fromInfo.setOnlineTablets(fromInfo.getOnlineTablets() - 1);
-        }
-        TableInfo toInfo = getTableInfo(currentCopy, serverTableIdCopied,
-            migration.tablet.tableId().toString(), migration.newServer);
-        if (toInfo != null) {
-          toInfo.setOnlineTablets(toInfo.getOnlineTablets() + 1);
-        }
-      }
-      migrations = EMPTY_MIGRATIONS;
-    } else {
-      migrationsFromLastPass.clear();
-    }
-
-    for (String s : tableIdMap.values()) {
-      TableId tableId = TableId.of(s);
-      String tableName = tableIdToTableName.get(tableId);
-      String regexTableName = getPoolNameForTable(tableName);
-      SortedMap<TServerInstance,TabletServerStatus> currentView =
-          currentGrouped.get(regexTableName);
-      if (currentView == null) {
-        LOG.warn("Skipping balance for table {} as no tablet servers are online.", tableName);
-        continue;
-      }
-      ArrayList<TabletMigration> newMigrations = new ArrayList<>();
-      getBalancerForTable(tableId).balance(currentView, migrations, newMigrations);
-
-      if (newMigrations.isEmpty()) {
-        tableToTimeSinceNoMigrations.remove(s);
-      } else if (tableToTimeSinceNoMigrations.containsKey(s)) {
-        if ((now - tableToTimeSinceNoMigrations.get(s)) > HOURS.toMillis(1)) {
-          LOG.warn("We have been consistently producing migrations for {}: {}", tableName,
-              limitTen(newMigrations));
-        }
-      } else {
-        tableToTimeSinceNoMigrations.put(s, now);
-      }
-
-      migrationsOut.addAll(newMigrations);
-      if (migrationsOut.size() >= myConf.maxTServerMigrations) {
-        break;
-      }
-    }
-
-    for (TabletMigration migration : migrationsOut) {
-      migrationsFromLastPass.put(migration.tablet, migration);
-    }
-
-    LOG.info("Migrating tablets for balance: {}", migrationsOut);
-    return minBalanceTime;
-  }
-
-  /**
-   * Get a mutable table info for the specified table and server
-   */
-  private TableInfo getTableInfo(SortedMap<TServerInstance,TabletServerStatus> currentCopy,
-      Multimap<TServerInstance,String> serverTableIdCopied, String tableId,
-      TServerInstance server) {
-    TableInfo newInfo = null;
-    if (currentCopy.containsKey(server)) {
-      Map<String,TableInfo> newTableMap = currentCopy.get(server).getTableMap();
-      if (newTableMap != null) {
-        newInfo = newTableMap.get(tableId);
-        if (newInfo != null) {
-          Collection<String> tableIdCopied = serverTableIdCopied.get(server);
-          if (tableIdCopied.isEmpty()) {
-            newTableMap = new HashMap<>(newTableMap);
-            currentCopy.get(server).setTableMap(newTableMap);
-          }
-          if (!tableIdCopied.contains(tableId)) {
-            newInfo = new TableInfo(newInfo);
-            newTableMap.put(tableId, newInfo);
-            tableIdCopied.add(tableId);
-          }
-        }
-      }
-    }
-    return newInfo;
-  }
-
-  // helper to prepare log messages
-  private static String limitTen(Collection<?> iterable) {
-    return iterable.stream().limit(10).map(String::valueOf)
-        .collect(Collectors.joining(", ", "[", "]"));
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.java
deleted file mode 100644
index f34608b..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/RegexGroupBalancer.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.server.master.balancer;
-
-import java.util.Map;
-import java.util.function.Function;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.hadoop.io.Text;
-
-/**
- * A {@link GroupBalancer} that groups tablets using a configurable regex. To use this balancer
- * configure the following settings for your table then configure this balancer for your table.
- *
- * <ul>
- * <li>Set {@code table.custom.balancer.group.regex.pattern} to a regular expression. This regular
- * expression must have one group. The regex is applied to the tablet end row and whatever the regex
- * group matches is used as the group. For example with a regex of {@code (\d\d).*} and an end row
- * of {@code 12abc}, the group for the tablet would be {@code 12}.
- * <li>Set {@code table.custom.balancer.group.regex.default} to a default group. This group is
- * returned for the last tablet in the table and tablets for which the regex does not match.
- * <li>Optionally set {@code table.custom.balancer.group.regex.wait.time} to time (can use time
- * suffixes). This determines how long to wait between balancing. Since this balancer scans the
- * metadata table, may want to set this higher for large tables.
- * </ul>
- *
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.RegexGroupBalancer}
- *             instead.
- */
-@Deprecated(since = "2.1.0")
-public class RegexGroupBalancer extends GroupBalancer {
-
-  public static final String REGEX_PROPERTY =
-      Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.group.regex.pattern";
-  public static final String DEFAUT_GROUP_PROPERTY =
-      Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.group.regex.default";
-  public static final String WAIT_TIME_PROPERTY =
-      Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.group.regex.wait.time";
-
-  private final TableId tableId;
-
-  public RegexGroupBalancer(TableId tableId) {
-    super(tableId);
-    this.tableId = tableId;
-  }
-
-  @Override
-  protected long getWaitTime() {
-    Map<String,String> customProps = context.getTableConfiguration(tableId)
-        .getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
-    if (customProps.containsKey(WAIT_TIME_PROPERTY)) {
-      return ConfigurationTypeHelper.getTimeInMillis(customProps.get(WAIT_TIME_PROPERTY));
-    }
-
-    return super.getWaitTime();
-  }
-
-  @Override
-  protected Function<KeyExtent,String> getPartitioner() {
-
-    Map<String,String> customProps = context.getTableConfiguration(tableId)
-        .getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
-    String regex = customProps.get(REGEX_PROPERTY);
-    final String defaultGroup = customProps.get(DEFAUT_GROUP_PROPERTY);
-
-    final Pattern pattern = Pattern.compile(regex);
-
-    return new Function<>() {
-
-      @Override
-      public String apply(KeyExtent input) {
-        Text er = input.endRow();
-        if (er == null) {
-          return defaultGroup;
-        }
-
-        Matcher matcher = pattern.matcher(er.toString());
-        if (matcher.matches() && matcher.groupCount() == 1) {
-          return matcher.group(1);
-        }
-
-        return defaultGroup;
-      }
-    };
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
deleted file mode 100644
index 3a42ae0..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
+++ /dev/null
@@ -1,166 +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.server.master.balancer;
-
-import java.lang.reflect.Constructor;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.classloader.ClassLoaderUtil;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.TableLoadBalancer}
- *             instead.
- */
-@Deprecated(since = "2.1.0")
-public class TableLoadBalancer extends TabletBalancer {
-
-  private static final Logger log = LoggerFactory.getLogger(TableLoadBalancer.class);
-
-  Map<TableId,TabletBalancer> perTableBalancers = new HashMap<>();
-
-  public TableLoadBalancer() {
-    log.warn(
-        "{} has been deprecated and will be removed in a future release. Please update your "
-            + "configuration to use the equivalent {} instead.",
-        getClass().getName(),
-        org.apache.accumulo.core.spi.balancer.TableLoadBalancer.class.getName());
-  }
-
-  private TabletBalancer constructNewBalancerForTable(String clazzName, TableId tableId)
-      throws Exception {
-    String context = null;
-    context = ClassLoaderUtil.tableContext(this.context.getTableConfiguration(tableId));
-    Class<? extends TabletBalancer> clazz =
-        ClassLoaderUtil.loadClass(context, clazzName, TabletBalancer.class);
-    Constructor<? extends TabletBalancer> constructor = clazz.getConstructor(TableId.class);
-    return constructor.newInstance(tableId);
-  }
-
-  protected String getLoadBalancerClassNameForTable(TableId table) {
-    TableState tableState = context.getTableManager().getTableState(table);
-    if (tableState == TableState.ONLINE) {
-      return this.context.getTableConfiguration(table).get(Property.TABLE_LOAD_BALANCER);
-    }
-    return null;
-  }
-
-  protected TabletBalancer getBalancerForTable(TableId tableId) {
-    TabletBalancer balancer = perTableBalancers.get(tableId);
-
-    String clazzName = getLoadBalancerClassNameForTable(tableId);
-
-    if (clazzName == null) {
-      clazzName = DefaultLoadBalancer.class.getName();
-    }
-    if (balancer != null) {
-      if (!clazzName.equals(balancer.getClass().getName())) {
-        // the balancer class for this table does not match the class specified in the configuration
-        try {
-          // attempt to construct a balancer with the specified class
-          TabletBalancer newBalancer = constructNewBalancerForTable(clazzName, tableId);
-          if (newBalancer != null) {
-            balancer = newBalancer;
-            perTableBalancers.put(tableId, balancer);
-            balancer.init(this.context);
-          }
-
-          log.info("Loaded new class {} for table {}", clazzName, tableId);
-        } catch (Exception e) {
-          log.warn("Failed to load table balancer class {} for table {}", clazzName, tableId, e);
-        }
-      }
-    }
-    if (balancer == null) {
-      try {
-        balancer = constructNewBalancerForTable(clazzName, tableId);
-        log.info("Loaded class {} for table {}", clazzName, tableId);
-      } catch (Exception e) {
-        log.warn("Failed to load table balancer class {} for table {}", clazzName, tableId, e);
-      }
-
-      if (balancer == null) {
-        log.info("Using balancer {} for table {}", DefaultLoadBalancer.class.getName(), tableId);
-        balancer = new DefaultLoadBalancer(tableId);
-      }
-      perTableBalancers.put(tableId, balancer);
-      balancer.init(this.context);
-    }
-    return balancer;
-  }
-
-  @Override
-  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
-      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments) {
-    // separate the unassigned into tables
-    Map<TableId,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<>();
-    unassigned.forEach((ke, lastTserver) -> groupedUnassigned
-        .computeIfAbsent(ke.tableId(), k -> new HashMap<>()).put(ke, lastTserver));
-    for (Entry<TableId,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
-      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<>();
-      getBalancerForTable(e.getKey()).getAssignments(current, e.getValue(), newAssignments);
-      assignments.putAll(newAssignments);
-    }
-  }
-
-  private TableOperations tops = null;
-
-  protected TableOperations getTableOperations() {
-    if (tops == null) {
-      tops = this.context.tableOperations();
-    }
-    return tops;
-  }
-
-  @Override
-  public long balance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
-    long minBalanceTime = 5_000;
-    // Iterate over the tables and balance each of them
-    TableOperations t = getTableOperations();
-    if (t == null) {
-      return minBalanceTime;
-    }
-    for (String s : t.tableIdMap().values()) {
-      ArrayList<TabletMigration> newMigrations = new ArrayList<>();
-      long tableBalanceTime =
-          getBalancerForTable(TableId.of(s)).balance(current, migrations, newMigrations);
-      if (tableBalanceTime < minBalanceTime) {
-        minBalanceTime = tableBalanceTime;
-      }
-      migrationsOut.addAll(newMigrations);
-    }
-    return minBalanceTime;
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
deleted file mode 100644
index 6777d93..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ /dev/null
@@ -1,301 +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.server.master.balancer;
-
-import static java.util.stream.Collectors.toList;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
-import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.manager.balancer.BalancerEnvironmentImpl;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This class is responsible for managing the distribution of tablets throughout an Accumulo
- * cluster. In most cases, users will want a balancer implementation which ensures a uniform
- * distribution of tablets, so that no individual tablet server is handling significantly more work
- * than any other.
- *
- * <p>
- * Implementations may wish to store configuration in Accumulo's system configuration using the
- * {@link Property#GENERAL_ARBITRARY_PROP_PREFIX}. They may also benefit from using per-table
- * configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
- *
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.core.spi.balancer.TabletBalancer}
- *             instead.
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
-    justification = "Class is deprecated and will be removed.")
-public abstract class TabletBalancer
-    implements org.apache.accumulo.core.spi.balancer.TabletBalancer {
-
-  private static final Logger log = LoggerFactory.getLogger(TabletBalancer.class);
-
-  protected ServerContext context;
-
-  @Override
-  public void init(BalancerEnvironment balancerEnvironment) {
-    var bei = (BalancerEnvironmentImpl) balancerEnvironment;
-    init(bei.getContext());
-  }
-
-  @Override
-  public void getAssignments(AssignmentParameters params) {
-    AssignmentParamsImpl api = (AssignmentParamsImpl) params;
-    getAssignments(api.thriftCurrentStatus(), api.thriftUnassigned(), api.thriftAssignmentsOut());
-  }
-
-  @Override
-  public long balance(BalanceParameters params) {
-    BalanceParamsImpl bpi = (BalanceParamsImpl) params;
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    long result = balance(bpi.thriftCurrentStatus(), bpi.thriftCurrentMigrations(), migrationsOut);
-    migrationsOut.forEach(mo -> bpi.addMigration(mo.tablet, mo.oldServer, mo.newServer));
-    return result;
-  }
-
-  /**
-   * Initialize the TabletBalancer. This gives the balancer the opportunity to read the
-   * configuration.
-   *
-   * @deprecated since 2.0.0; use {@link #init(ServerContext)} instead.
-   */
-  @Deprecated(since = "2.0.0")
-  public void init(ServerConfigurationFactory conf) {
-    init(conf.getServerContext());
-  }
-
-  /**
-   * Initialize the TabletBalancer. This gives the balancer the opportunity to read the
-   * configuration.
-   *
-   * @since 2.0.0
-   */
-  public void init(ServerContext context) {
-    this.context = context;
-  }
-
-  /**
-   * Assign tablets to tablet servers. This method is called whenever the manager finds tablets that
-   * are unassigned.
-   *
-   * @param current The current table-summary state of all the online tablet servers. Read-only. The
-   *        TabletServerStatus for each server may be null if the tablet server has not yet
-   *        responded to a recent request for status.
-   * @param unassigned A map from unassigned tablet to the last known tablet server. Read-only.
-   * @param assignments A map from tablet to assigned server. Write-only.
-   */
-  public abstract void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current,
-      Map<KeyExtent,TServerInstance> unassigned, Map<KeyExtent,TServerInstance> assignments);
-
-  /**
-   * Ask the balancer if any migrations are necessary.
-   *
-   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires
-   * some minimum number of tservers, or a maximum number of outstanding migrations), it should
-   * issue a log message to alert operators. The message should be at WARN normally and at ERROR if
-   * the balancer knows that the problem can not self correct. It should not issue these messages
-   * more than once a minute.
-   *
-   * @param current The current table-summary state of all the online tablet servers. Read-only.
-   * @param migrations the current set of migrations. Read-only.
-   * @param migrationsOut new migrations to perform; should not contain tablets in the current set
-   *        of migrations. Write-only.
-   * @return the time, in milliseconds, to wait before re-balancing.
-   *
-   *         This method will not be called when there are unassigned tablets.
-   */
-  public abstract long balance(SortedMap<TServerInstance,TabletServerStatus> current,
-      Set<KeyExtent> migrations, List<TabletMigration> migrationsOut);
-
-  private static final long ONE_SECOND = 1000L;
-  private boolean stuck = false;
-  private long stuckNotificationTime = -1L;
-
-  protected static final long TIME_BETWEEN_BALANCER_WARNINGS = 60 * ONE_SECOND;
-
-  /**
-   * A deferred call descendent TabletBalancers use to log why they can't continue. The call is
-   * deferred so that TabletBalancer can limit how often messages happen.
-   *
-   * Implementations should be reused as much as possible.
-   *
-   * Be sure to pass in a properly scoped Logger instance so that messages indicate what part of the
-   * system is having trouble.
-   */
-  protected abstract static class BalancerProblem implements Runnable {
-    protected final Logger balancerLog;
-
-    public BalancerProblem(Logger logger) {
-      balancerLog = logger;
-    }
-  }
-
-  /**
-   * If a TabletBalancer requires active tservers, it should use this problem to indicate when there
-   * are none. NoTservers is safe to share with anyone who uses the same Logger. TabletBalancers
-   * should have a single static instance.
-   */
-  protected static class NoTservers extends BalancerProblem {
-    public NoTservers(Logger logger) {
-      super(logger);
-    }
-
-    @Override
-    public void run() {
-      balancerLog.warn("Not balancing because we don't have any tservers");
-    }
-  }
-
-  /**
-   * If a TabletBalancer only balances when there are no outstanding migrations, it should use this
-   * problem to indicate when they exist.
-   *
-   * Iff a TabletBalancer makes use of the migrations member to provide samples, then
-   * OutstandingMigrations is not thread safe.
-   */
-  protected static class OutstandingMigrations extends BalancerProblem {
-    public Set<KeyExtent> migrations = Collections.emptySet();
-
-    public OutstandingMigrations(Logger logger) {
-      super(logger);
-    }
-
-    @Override
-    public void run() {
-      balancerLog.warn("Not balancing due to {} outstanding migrations.", migrations.size());
-      balancerLog.debug("Sample up to 10 outstanding migrations: {}",
-          migrations.stream().limit(10).collect(toList()));
-    }
-  }
-
-  /**
-   * Warn that a Balancer can't work because of some external restriction. Will not call the
-   * provided logging handler more often than TIME_BETWEEN_BALANCER_WARNINGS
-   */
-  protected void constraintNotMet(BalancerProblem cause) {
-    if (stuck) {
-      if ((System.currentTimeMillis() - stuckNotificationTime) > TIME_BETWEEN_BALANCER_WARNINGS) {
-        cause.run();
-        stuckNotificationTime = System.currentTimeMillis();
-      }
-    } else {
-      stuck = true;
-      stuckNotificationTime = System.currentTimeMillis();
-    }
-  }
-
-  /**
-   * Resets logging about problems meeting an external constraint on balancing.
-   */
-  protected void resetBalancerErrors() {
-    stuck = false;
-  }
-
-  /**
-   * Fetch the tablets for the given table by asking the tablet server. Useful if your balance
-   * strategy needs details at the tablet level to decide what tablets to move.
-   *
-   * @param tserver The tablet server to ask.
-   * @param tableId The table id
-   * @return a list of tablet statistics
-   * @throws ThriftSecurityException tablet server disapproves of your internal System password.
-   * @throws TException any other problem
-   */
-  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId)
-      throws ThriftSecurityException, TException {
-    log.debug("Scanning tablet server {} for table {}", tserver, tableId);
-    Client client =
-        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, tserver.getHostAndPort(), context);
-    try {
-      return client.getTabletStats(TraceUtil.traceInfo(), context.rpcCreds(), tableId.canonical());
-    } catch (TTransportException e) {
-      log.error("Unable to connect to {}: ", tserver, e);
-    } finally {
-      ThriftUtil.returnClient(client, context);
-    }
-    return null;
-  }
-
-  /**
-   * Utility to ensure that the migrations from balance() are consistent:
-   * <ul>
-   * <li>Tablet objects are not null
-   * <li>Source and destination tablet servers are not null and current
-   * </ul>
-   *
-   * @return A list of TabletMigration object that passed sanity checks.
-   */
-  public static List<TabletMigration> checkMigrationSanity(Set<TServerInstance> current,
-      List<TabletMigration> migrations) {
-    List<TabletMigration> result = new ArrayList<>(migrations.size());
-    for (TabletMigration m : migrations) {
-      if (m.tablet == null) {
-        log.warn("Balancer gave back a null tablet {}", m);
-        continue;
-      }
-      if (m.newServer == null) {
-        log.warn("Balancer did not set the destination {}", m);
-        continue;
-      }
-      if (m.oldServer == null) {
-        log.warn("Balancer did not set the source {}", m);
-        continue;
-      }
-      if (!current.contains(m.oldServer)) {
-        log.warn("Balancer wants to move a tablet from a server that is not current: {}", m);
-        continue;
-      }
-      if (!current.contains(m.newServer)) {
-        log.warn("Balancer wants to move a tablet to a server that is not current: {}", m);
-        continue;
-      }
-      result.add(m);
-    }
-    return result;
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
deleted file mode 100644
index 036b402..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
+++ /dev/null
@@ -1,42 +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.server.master.recovery;
-
-import org.apache.accumulo.core.conf.Property;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.server.manager.recovery.HadoopLogCloser}
- *             instead
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Compatibility class to be removed in next major release.")
-public class HadoopLogCloser extends org.apache.accumulo.server.manager.recovery.HadoopLogCloser {
-  private static final Logger log = LoggerFactory.getLogger(HadoopLogCloser.class);
-
-  public HadoopLogCloser() {
-    log.warn("{} has been deprecated. Please update property {} to {} instead.",
-        getClass().getName(), Property.MANAGER_WAL_CLOSER_IMPLEMENTATION.getKey(),
-        org.apache.accumulo.server.manager.recovery.HadoopLogCloser.class.getName());
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.java
deleted file mode 100644
index 132e22d..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletMigration.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.server.master.state;
-
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.TServerInstance;
-
-/**
- * @deprecated since 2.1.0. Use balancers in org.apache.accumulo.core.spi.balancer instead.
- */
-@Deprecated(since = "2.1.0")
-public class TabletMigration {
-  public KeyExtent tablet;
-  public TServerInstance oldServer;
-  public TServerInstance newServer;
-
-  public TabletMigration(KeyExtent extent, TServerInstance before, TServerInstance after) {
-    this.tablet = extent;
-    this.oldServer = before;
-    this.newServer = after;
-  }
-
-  @Override
-  public String toString() {
-    return tablet + ": " + oldServer + " -> " + newServer;
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/mem/LowMemoryDetector.java b/server/base/src/main/java/org/apache/accumulo/server/mem/LowMemoryDetector.java
new file mode 100644
index 0000000..fb18c7c
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/mem/LowMemoryDetector.java
@@ -0,0 +1,200 @@
+/*
+ * 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.server.mem;
+
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.server.ServerContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LowMemoryDetector {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LowMemoryDetector.class);
+
+  @FunctionalInterface
+  public interface Action {
+    void execute();
+  }
+
+  public enum DetectionScope {
+    MINC, MAJC, SCAN
+  }
+
+  private final HashMap<String,Long> prevGcTime = new HashMap<>();
+
+  private long lastMemorySize = 0;
+  private int lowMemCount = 0;
+  private long lastMemoryCheckTime = 0;
+  private final Lock memCheckTimeLock = new ReentrantLock();
+  private volatile boolean runningLowOnMemory = false;
+
+  public long getIntervalMillis(AccumuloConfiguration conf) {
+    return conf.getTimeInMillis(Property.GENERAL_LOW_MEM_DETECTOR_INTERVAL);
+  }
+
+  public boolean isRunningLowOnMemory() {
+    return runningLowOnMemory;
+  }
+
+  /**
+   * @param context server context
+   * @param scope whether this is being checked in the context of scan or compact code
+   * @param isUserTable boolean set true if the table being scanned / compacted is a user table. No
+   *        action is taken for system tables.
+   * @param action Action to perform when this method returns true
+   * @return true if server running low on memory
+   */
+  public boolean isRunningLowOnMemory(ServerContext context, DetectionScope scope,
+      Supplier<Boolean> isUserTable, Action action) {
+    if (isUserTable.get()) {
+      Property p;
+      switch (scope) {
+        case SCAN:
+          p = Property.GENERAL_LOW_MEM_SCAN_PROTECTION;
+          break;
+        case MINC:
+          p = Property.GENERAL_LOW_MEM_MINC_PROTECTION;
+          break;
+        case MAJC:
+          p = Property.GENERAL_LOW_MEM_MAJC_PROTECTION;
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown scope: " + scope);
+      }
+      boolean isEnabled = context.getConfiguration().getBoolean(p);
+      // Only incur the penalty of accessing the volatile variable when enabled for this scope
+      if (isEnabled && runningLowOnMemory) {
+        action.execute();
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public void logGCInfo(AccumuloConfiguration conf) {
+
+    double freeMemoryPercentage = conf.getFraction(Property.GENERAL_LOW_MEM_DETECTOR_THRESHOLD);
+
+    memCheckTimeLock.lock();
+    try {
+      final long now = TimeUnit.NANOSECONDS.toMillis(System.nanoTime());
+
+      List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
+
+      StringBuilder sb = new StringBuilder("gc");
+
+      boolean sawChange = false;
+
+      long maxIncreaseInCollectionTime = 0;
+      for (GarbageCollectorMXBean gcBean : gcmBeans) {
+        Long prevTime = prevGcTime.get(gcBean.getName());
+        long pt = 0;
+        if (prevTime != null) {
+          pt = prevTime;
+        }
+
+        long time = gcBean.getCollectionTime();
+
+        if (time - pt != 0) {
+          sawChange = true;
+        }
+
+        long increaseInCollectionTime = time - pt;
+        sb.append(String.format(" %s=%,.2f(+%,.2f) secs", gcBean.getName(), time / 1000.0,
+            increaseInCollectionTime / 1000.0));
+        maxIncreaseInCollectionTime =
+            Math.max(increaseInCollectionTime, maxIncreaseInCollectionTime);
+        prevGcTime.put(gcBean.getName(), time);
+      }
+
+      Runtime rt = Runtime.getRuntime();
+      final long maxConfiguredMemory = rt.maxMemory();
+      final long allocatedMemory = rt.totalMemory();
+      final long allocatedFreeMemory = rt.freeMemory();
+      final long freeMemory = maxConfiguredMemory - (allocatedMemory - allocatedFreeMemory);
+      final long lowMemoryThreshold = (long) (maxConfiguredMemory * freeMemoryPercentage);
+      LOG.trace("Memory info: max={}, allocated={}, free={}, free threshold={}",
+          maxConfiguredMemory, allocatedMemory, freeMemory, lowMemoryThreshold);
+
+      if (freeMemory < lowMemoryThreshold) {
+        lowMemCount++;
+        if (lowMemCount > 3 && !runningLowOnMemory) {
+          runningLowOnMemory = true;
+          LOG.warn("Running low on memory: max={}, allocated={}, free={}, free threshold={}",
+              maxConfiguredMemory, allocatedMemory, freeMemory, lowMemoryThreshold);
+        }
+      } else {
+        // If we were running low on memory, but are not any longer, than log at warn
+        // so that it shows up in the logs
+        if (runningLowOnMemory) {
+          LOG.warn("Recovered from low memory condition");
+        } else {
+          LOG.trace("Not running low on memory");
+        }
+        runningLowOnMemory = false;
+        lowMemCount = 0;
+      }
+
+      if (freeMemory != lastMemorySize) {
+        sawChange = true;
+      }
+
+      sb.append(String.format(" freemem=%,d(%+,d) totalmem=%,d", freeMemory,
+          (freeMemory - lastMemorySize), rt.totalMemory()));
+
+      if (sawChange) {
+        LOG.debug(sb.toString());
+      }
+
+      final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
+      if (lastMemoryCheckTime > 0 && lastMemoryCheckTime < now) {
+        final long diff = now - lastMemoryCheckTime;
+        if (diff > keepAliveTimeout + 1000) {
+          LOG.warn(String.format(
+              "GC pause checker not called in a timely"
+                  + " fashion. Expected every %.1f seconds but was %.1f seconds since last check",
+              keepAliveTimeout / 1000., diff / 1000.));
+        }
+        lastMemoryCheckTime = now;
+        return;
+      }
+
+      if (maxIncreaseInCollectionTime > keepAliveTimeout) {
+        Halt.halt("Garbage collection may be interfering with lock keep-alive. Halting.", -1);
+      }
+
+      lastMemorySize = freeMemory;
+      lastMemoryCheckTime = now;
+    } finally {
+      memCheckTimeLock.unlock();
+    }
+  }
+
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootGcCandidates.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootGcCandidates.java
index 13d0246..b579329 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootGcCandidates.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootGcCandidates.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.server.metadata;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 
 import java.security.SecureRandom;
 import java.util.SortedMap;
@@ -31,13 +32,10 @@
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.hadoop.fs.Path;
 
-import com.google.gson.Gson;
-
 public class RootGcCandidates {
   // Version 1. Released with Accumulo version 2.1.0
   private static final int VERSION = 1;
 
-  private final Gson gson = new Gson();
   private final Data data;
 
   // This class is used to serialize and deserialize root tablet metadata using GSon. Any changes to
@@ -65,7 +63,7 @@
   }
 
   public RootGcCandidates(String jsonString) {
-    this.data = gson.fromJson(jsonString, Data.class);
+    this.data = GSON.get().fromJson(jsonString, Data.class);
     checkArgument(data.version == VERSION, "Invalid Root Table GC Candidates JSON version %s",
         data.version);
     data.candidates.forEach((parent, files) -> {
@@ -100,7 +98,7 @@
   }
 
   public String toJson() {
-    return gson.toJson(data);
+    return GSON.get().toJson(data);
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
index 73f91b0..7ebdd7d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/RootTabletMutatorImpl.java
@@ -24,7 +24,7 @@
 
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TabletId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.data.constraints.Constraint;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.Ample;
@@ -41,9 +41,7 @@
 
   private static final Logger log = LoggerFactory.getLogger(RootTabletMutatorImpl.class);
 
-  @SuppressWarnings("deprecation")
-  private static class RootEnv
-      implements SystemEnvironment, org.apache.accumulo.core.constraints.Constraint.Environment {
+  private static class RootEnv implements SystemEnvironment, Constraint.Environment {
 
     private final ServerContext context;
 
@@ -52,11 +50,6 @@
     }
 
     @Override
-    public KeyExtent getExtent() {
-      return RootTable.EXTENT;
-    }
-
-    @Override
     public TabletId getTablet() {
       return new TabletIdImpl(RootTable.EXTENT);
     }
@@ -117,7 +110,7 @@
         closeAfterMutate.close();
       }
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
index ef02f22..1955402 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java
@@ -22,6 +22,7 @@
 import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET_GC_CANDIDATES;
 import static org.apache.accumulo.server.util.MetadataTableUtil.EMPTY_TEXT;
 
+import java.net.URI;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
@@ -47,7 +48,7 @@
 import org.apache.accumulo.core.fate.FateTxId;
 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.AccumuloTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
@@ -114,14 +115,14 @@
         return newJson.getBytes(UTF_8);
       });
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
   @Override
   public void putGcCandidates(TableId tableId, Collection<StoredTabletFile> candidates) {
 
-    if (RootTable.ID.equals(tableId)) {
+    if (AccumuloTable.ROOT.tableId().equals(tableId)) {
       mutateRootGcCandidates(rgcc -> rgcc.add(candidates.stream()));
       return;
     }
@@ -131,7 +132,7 @@
         writer.addMutation(createDeleteMutation(file));
       }
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -141,7 +142,7 @@
     if (DataLevel.of(tableId) == DataLevel.ROOT) {
       // Directories are unexpected for the root tablet, so convert to stored tablet file
       mutateRootGcCandidates(rgcc -> rgcc.add(candidates.stream()
-          .map(reference -> new StoredTabletFile(reference.getMetadataEntry()))));
+          .map(reference -> StoredTabletFile.of(URI.create(reference.getMetadataPath())))));
       return;
     }
 
@@ -150,7 +151,7 @@
         writer.addMutation(createDeleteMutation(fileOrDir));
       }
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -162,10 +163,10 @@
     Mutation m = new Mutation(BlipSection.getRowPrefix() + path);
     m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(FateTxId.formatTid(fateTxid)));
 
-    try (BatchWriter bw = context.createBatchWriter(MetadataTable.NAME)) {
+    try (BatchWriter bw = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       bw.addMutation(m);
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -177,10 +178,10 @@
     Mutation m = new Mutation(BlipSection.getRowPrefix() + path);
     m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 
-    try (BatchWriter bw = context.createBatchWriter(MetadataTable.NAME)) {
+    try (BatchWriter bw = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       bw.addMutation(m);
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -188,9 +189,9 @@
   public void removeBulkLoadEntries(TableId tableId, long tid, Text firstSplit, Text lastSplit) {
     Preconditions.checkArgument(DataLevel.of(tableId) == DataLevel.USER);
     try (
-        Scanner mscanner =
-            new IsolatedScanner(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
-        BatchWriter bw = context.createBatchWriter(MetadataTable.NAME)) {
+        Scanner mscanner = new IsolatedScanner(
+            context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY));
+        BatchWriter bw = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       mscanner.setRange(new KeyExtent(tableId, lastSplit, firstSplit).toMetaRange());
       mscanner.fetchColumnFamily(BulkFileColumnFamily.NAME);
 
@@ -206,7 +207,7 @@
         }
       }
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -242,7 +243,7 @@
         }
       }
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -255,7 +256,7 @@
         jsonBytes =
             zooReader.getData(context.getZooKeeperRoot() + RootTable.ZROOT_TABLET_GC_CANDIDATES);
       } catch (KeeperException | InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
       return new RootGcCandidates(new String(jsonBytes, UTF_8)).sortedStream().iterator();
     } else if (level == DataLevel.METADATA || level == DataLevel.USER) {
@@ -265,7 +266,7 @@
       try {
         scanner = context.createScanner(level.metaTable(), Authorizations.EMPTY);
       } catch (TableNotFoundException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
       scanner.setRange(range);
       return scanner.stream().filter(entry -> entry.getValue().equals(SkewedKeyValue.NAME))
@@ -280,11 +281,11 @@
 
   @Override
   public Mutation createDeleteMutation(ReferenceFile tabletFilePathToRemove) {
-    return createDelMutation(ValidationUtil.validate(tabletFilePathToRemove).getMetadataEntry());
+    return createDelMutation(ValidationUtil.validate(tabletFilePathToRemove).getMetadataPath());
   }
 
   public Mutation createDeleteMutation(StoredTabletFile pathToRemove) {
-    return createDelMutation(pathToRemove.getMetaUpdateDelete());
+    return createDelMutation(pathToRemove.getMetadataPath());
   }
 
   private Mutation createDelMutation(String path) {
@@ -304,7 +305,7 @@
         writer.addMutation(m);
       }
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -314,7 +315,7 @@
     try {
       scanner = context.createScanner(DataLevel.USER.metaTable(), Authorizations.EMPTY);
     } catch (TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
 
     scanner.setRange(ExternalCompactionSection.getRange());
@@ -338,7 +339,7 @@
       log.debug("Deleted external compaction final state entries for external compactions: {}",
           statusesToDelete);
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -363,7 +364,7 @@
       Scanner scanner = context.createScanner(DataLevel.USER.metaTable(), Authorizations.EMPTY);
       scanner.setRange(ScanServerFileReferenceSection.getRange());
       int pLen = ScanServerFileReferenceSection.getRowPrefix().length();
-      return StreamSupport.stream(scanner.spliterator(), false)
+      return scanner.stream().onClose(scanner::close)
           .map(e -> new ScanServerRefTabletFile(e.getKey().getRowData().toString().substring(pLen),
               e.getKey().getColumnFamily(), e.getKey().getColumnQualifier()));
     } catch (TableNotFoundException e) {
@@ -405,7 +406,7 @@
       }
       log.debug("Deleted scan server file reference entries for files: {}", refsToDelete);
     } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
index 2987a44..a8b5677 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorBase.java
@@ -22,24 +22,22 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.FateTxId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 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.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
 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.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 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.ScanFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
@@ -48,6 +46,7 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
 
@@ -81,38 +80,38 @@
   }
 
   @Override
-  public Ample.TabletMutator putFile(TabletFile path, DataFileValue dfv) {
+  public Ample.TabletMutator putFile(ReferencedTabletFile path, DataFileValue dfv) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.put(DataFileColumnFamily.NAME, path.getMetaInsertText(), new Value(dfv.encode()));
+    mutation.put(DataFileColumnFamily.NAME, path.insert().getMetadataText(),
+        new Value(dfv.encode()));
     return this;
   }
 
   @Override
   public Ample.TabletMutator putFile(StoredTabletFile path, DataFileValue dfv) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.put(DataFileColumnFamily.NAME, path.getMetaUpdateDeleteText(),
-        new Value(dfv.encode()));
+    mutation.put(DataFileColumnFamily.NAME, path.getMetadataText(), new Value(dfv.encode()));
     return this;
   }
 
   @Override
   public Ample.TabletMutator deleteFile(StoredTabletFile path) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.putDelete(DataFileColumnFamily.NAME, path.getMetaUpdateDeleteText());
+    mutation.putDelete(DataFileColumnFamily.NAME, path.getMetadataText());
     return this;
   }
 
   @Override
-  public Ample.TabletMutator putScan(TabletFile path) {
+  public Ample.TabletMutator putScan(StoredTabletFile path) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.put(ScanFileColumnFamily.NAME, path.getMetaInsertText(), new Value());
+    mutation.put(ScanFileColumnFamily.NAME, path.getMetadataText(), new Value());
     return this;
   }
 
   @Override
   public Ample.TabletMutator deleteScan(StoredTabletFile path) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.putDelete(ScanFileColumnFamily.NAME, path.getMetaUpdateDeleteText());
+    mutation.putDelete(ScanFileColumnFamily.NAME, path.getMetadataText());
     return this;
   }
 
@@ -176,48 +175,34 @@
   @Override
   public Ample.TabletMutator putWal(LogEntry logEntry) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
+    logEntry.addToMutation(mutation);
     return this;
   }
 
   @Override
   public Ample.TabletMutator deleteWal(LogEntry logEntry) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.putDelete(logEntry.getColumnFamily(), logEntry.getColumnQualifier());
+    logEntry.deleteFromMutation(mutation);
     return this;
   }
 
   @Override
-  public Ample.TabletMutator deleteWal(String wal) {
+  public Ample.TabletMutator putBulkFile(ReferencedTabletFile bulkref, long tid) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.putDelete(LogColumnFamily.STR_NAME, wal);
-    return this;
-  }
-
-  @Override
-  public Ample.TabletMutator putBulkFile(TabletFile bulkref, long tid) {
-    Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.put(BulkFileColumnFamily.NAME, bulkref.getMetaInsertText(),
+    mutation.put(BulkFileColumnFamily.NAME, bulkref.insert().getMetadataText(),
         new Value(FateTxId.formatTid(tid)));
     return this;
   }
 
   @Override
-  public Ample.TabletMutator deleteBulkFile(TabletFile bulkref) {
+  public Ample.TabletMutator deleteBulkFile(StoredTabletFile bulkref) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    mutation.putDelete(BulkFileColumnFamily.NAME, bulkref.getMetaInsertText());
+    mutation.putDelete(BulkFileColumnFamily.NAME, bulkref.getMetadataText());
     return this;
   }
 
   @Override
-  public Ample.TabletMutator putChopped() {
-    Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
-    ChoppedColumnFamily.CHOPPED_COLUMN.put(mutation, new Value("chopped"));
-    return this;
-  }
-
-  @Override
-  public Ample.TabletMutator putSuspension(TServerInstance tServer, long suspensionTime) {
+  public Ample.TabletMutator putSuspension(TServerInstance tServer, SteadyTime suspensionTime) {
     Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate.");
     mutation.put(SuspendLocationColumn.SUSPEND_COLUMN.getColumnFamily(),
         SuspendLocationColumn.SUSPEND_COLUMN.getColumnQualifier(),
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
index e488987..d6afd6d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletMutatorImpl.java
@@ -41,7 +41,7 @@
         closeAfterMutate.close();
       }
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletsMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletsMutatorImpl.java
index 8bac627..2ce5b66 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletsMutatorImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/TabletsMutatorImpl.java
@@ -23,8 +23,7 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 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.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletsMutator;
 import org.apache.accumulo.server.ServerContext;
@@ -44,24 +43,24 @@
 
   private BatchWriter getWriter(TableId tableId) {
 
-    Preconditions.checkArgument(!RootTable.ID.equals(tableId));
+    Preconditions.checkArgument(!AccumuloTable.ROOT.tableId().equals(tableId));
 
     try {
-      if (MetadataTable.ID.equals(tableId)) {
+      if (AccumuloTable.METADATA.tableId().equals(tableId)) {
         if (rootWriter == null) {
-          rootWriter = context.createBatchWriter(RootTable.NAME);
+          rootWriter = context.createBatchWriter(AccumuloTable.ROOT.tableName());
         }
 
         return rootWriter;
       } else {
         if (metaWriter == null) {
-          metaWriter = context.createBatchWriter(MetadataTable.NAME);
+          metaWriter = context.createBatchWriter(AccumuloTable.METADATA.tableName());
         }
 
         return metaWriter;
       }
     } catch (TableNotFoundException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -85,7 +84,7 @@
         metaWriter.close();
       }
     } catch (MutationsRejectedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
 
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
index 15322bc..988bf38 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
@@ -36,13 +36,13 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.metrics.MetricsProducer;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.server.ServerContext;
 import org.checkerframework.checker.nullness.qual.NonNull;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
 
 import io.micrometer.core.instrument.Meter;
 import io.micrometer.core.instrument.MeterRegistry;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/metrics/ProcessMetrics.java b/server/base/src/main/java/org/apache/accumulo/server/metrics/ProcessMetrics.java
new file mode 100644
index 0000000..8b44525
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/metrics/ProcessMetrics.java
@@ -0,0 +1,44 @@
+/*
+ * 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.server.metrics;
+
+import java.util.List;
+
+import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.server.ServerContext;
+
+import io.micrometer.core.instrument.MeterRegistry;
+
+public class ProcessMetrics implements MetricsProducer {
+
+  private final ServerContext context;
+
+  public ProcessMetrics(final ServerContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public void registerMetrics(MeterRegistry registry) {
+    registry.gauge(METRICS_LOW_MEMORY, List.of(), this, this::lowMemDetected);
+  }
+
+  private int lowMemDetected(ProcessMetrics processMetrics) {
+    return context.getLowMemoryDetector().isRunningLowOnMemory() ? 1 : 0;
+  }
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
index ab4a730..8f92056 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
@@ -37,7 +37,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ProblemSection;
 import org.apache.accumulo.core.util.Encoding;
 import org.apache.accumulo.server.ServerContext;
@@ -142,7 +142,7 @@
   void removeFromMetadataTable(ServerContext context) throws Exception {
     Mutation m = new Mutation(ProblemSection.getRowPrefix() + tableId);
     m.putDelete(problemType.name(), resource);
-    try (var writer = context.createBatchWriter(MetadataTable.NAME)) {
+    try (var writer = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       writer.addMutation(m);
     }
   }
@@ -150,7 +150,7 @@
   void saveToMetadataTable(ServerContext context) throws Exception {
     Mutation m = new Mutation(ProblemSection.getRowPrefix() + tableId);
     m.put(problemType.name(), resource, new Value(encode()));
-    try (var writer = context.createBatchWriter(MetadataTable.NAME)) {
+    try (var writer = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       writer.addMutation(m);
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
index 2c04fc9..2708d21 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
@@ -20,6 +20,8 @@
 
 import static java.util.concurrent.TimeUnit.SECONDS;
 
+import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collections;
 import java.util.EnumMap;
 import java.util.Iterator;
@@ -36,6 +38,7 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -44,13 +47,13 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.iterators.SortedKeyIterator;
-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.ProblemSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.commons.collections4.map.LRUMap;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -160,7 +163,8 @@
       return;
     }
 
-    Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner scanner =
+        context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
     scanner.addScanIterator(new IteratorSetting(1, "keys-only", SortedKeyIterator.class));
 
     scanner.setRange(new Range(ProblemSection.getRowPrefix() + table));
@@ -174,108 +178,106 @@
     }
 
     if (hasProblems) {
-      try (var writer = context.createBatchWriter(MetadataTable.NAME)) {
+      try (var writer = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
         writer.addMutation(delMut);
       }
     }
   }
 
   private static boolean isMeta(TableId tableId) {
-    return tableId.equals(MetadataTable.ID) || tableId.equals(RootTable.ID);
+    return tableId.equals(AccumuloTable.METADATA.tableId())
+        || tableId.equals(AccumuloTable.ROOT.tableId());
   }
 
   public Iterator<ProblemReport> iterator(final TableId table) {
-    try {
+    return new Iterator<>() {
 
-      return new Iterator<>() {
+      ZooReaderWriter zoo = context.getZooReaderWriter();
+      private int iter1Count = 0;
+      private Iterator<String> iter1;
 
-        ZooReaderWriter zoo = context.getZooReaderWriter();
-        private int iter1Count = 0;
-        private Iterator<String> iter1;
-
-        private Iterator<String> getIter1() {
-          if (iter1 == null) {
-            try {
-              List<String> children;
-              if (table == null || isMeta(table)) {
-                children = zoo.getChildren(context.getZooKeeperRoot() + Constants.ZPROBLEMS);
-              } else {
-                children = Collections.emptyList();
-              }
-              iter1 = children.iterator();
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          }
-
-          return iter1;
-        }
-
-        private Iterator<Entry<Key,Value>> iter2;
-
-        private Iterator<Entry<Key,Value>> getIter2() {
-          if (iter2 == null) {
-            try {
-              if ((table == null || !isMeta(table)) && iter1Count == 0) {
-                Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-                scanner.setTimeout(3, TimeUnit.SECONDS);
-
-                if (table == null) {
-                  scanner.setRange(ProblemSection.getRange());
-                } else {
-                  scanner.setRange(new Range(ProblemSection.getRowPrefix() + table));
-                }
-
-                iter2 = scanner.iterator();
-
-              } else {
-                Map<Key,Value> m = Collections.emptyMap();
-                iter2 = m.entrySet().iterator();
-              }
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          }
-
-          return iter2;
-        }
-
-        @Override
-        public boolean hasNext() {
-          if (getIter1().hasNext()) {
-            return true;
-          }
-          return getIter2().hasNext();
-        }
-
-        @Override
-        public ProblemReport next() {
+      private Iterator<String> getIter1() {
+        if (iter1 == null) {
           try {
-            if (getIter1().hasNext()) {
-              iter1Count++;
-              return ProblemReport.decodeZooKeeperEntry(context, getIter1().next());
+            List<String> children;
+            if (table == null || isMeta(table)) {
+              children = zoo.getChildren(context.getZooKeeperRoot() + Constants.ZPROBLEMS);
+            } else {
+              children = Collections.emptyList();
             }
+            iter1 = children.iterator();
+          } catch (KeeperException | InterruptedException e) {
+            throw new IllegalStateException(e);
+          }
+        }
 
-            if (getIter2().hasNext()) {
-              return ProblemReport.decodeMetadataEntry(getIter2().next());
+        return iter1;
+      }
+
+      private Iterator<Entry<Key,Value>> iter2;
+
+      private Iterator<Entry<Key,Value>> getIter2() {
+        if (iter2 == null) {
+          try {
+            if ((table == null || !isMeta(table)) && iter1Count == 0) {
+              Scanner scanner =
+                  context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
+              scanner.setTimeout(3, TimeUnit.SECONDS);
+
+              if (table == null) {
+                scanner.setRange(ProblemSection.getRange());
+              } else {
+                scanner.setRange(new Range(ProblemSection.getRowPrefix() + table));
+              }
+
+              iter2 = scanner.iterator();
+
+            } else {
+              Map<Key,Value> m = Collections.emptyMap();
+              iter2 = m.entrySet().iterator();
             }
-          } catch (Exception e) {
-            throw new RuntimeException(e);
+          } catch (TableNotFoundException e) {
+            throw new IllegalStateException(e);
+          }
+        }
+
+        return iter2;
+      }
+
+      @Override
+      public boolean hasNext() {
+        if (getIter1().hasNext()) {
+          return true;
+        }
+        return getIter2().hasNext();
+      }
+
+      @Override
+      public ProblemReport next() {
+        try {
+          if (getIter1().hasNext()) {
+            iter1Count++;
+            return ProblemReport.decodeZooKeeperEntry(context, getIter1().next());
           }
 
-          throw new NoSuchElementException();
+          if (getIter2().hasNext()) {
+            return ProblemReport.decodeMetadataEntry(getIter2().next());
+          }
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        } catch (KeeperException | InterruptedException e) {
+          throw new IllegalStateException(e);
         }
 
-        @Override
-        public void remove() {
-          throw new UnsupportedOperationException();
-        }
+        throw new NoSuchElementException();
+      }
 
-      };
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
 
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
+    };
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java b/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java
deleted file mode 100644
index b297c7f..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/DistributedWorkQueueWorkAssignerHelper.java
+++ /dev/null
@@ -1,81 +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.server.replication;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-
-import com.google.common.collect.Maps;
-
-@Deprecated
-public class DistributedWorkQueueWorkAssignerHelper {
-
-  public static final String KEY_SEPARATOR = "|";
-
-  /**
-   * Serialize a filename and a {@link ReplicationTarget} into the expected key format for use with
-   * the {@link DistributedWorkQueue}
-   *
-   * @param filename Filename for data to be replicated
-   * @param replTarget Information about replication peer
-   * @return Key for identifying work in queue
-   */
-  public static String getQueueKey(String filename, ReplicationTarget replTarget) {
-    return filename + KEY_SEPARATOR + replTarget.getPeerName() + KEY_SEPARATOR
-        + replTarget.getRemoteIdentifier() + KEY_SEPARATOR + replTarget.getSourceTableId();
-  }
-
-  /**
-   * @param queueKey Key from the work queue
-   * @return Components which created the queue key
-   */
-  public static Entry<String,ReplicationTarget> fromQueueKey(String queueKey) {
-    requireNonNull(queueKey);
-
-    int index = queueKey.indexOf(KEY_SEPARATOR);
-    if (index == -1) {
-      throw new IllegalArgumentException(
-          "Could not find expected separator in queue key '" + queueKey + "'");
-    }
-
-    String filename = queueKey.substring(0, index);
-
-    int secondIndex = queueKey.indexOf(KEY_SEPARATOR, index + 1);
-    if (secondIndex == -1) {
-      throw new IllegalArgumentException(
-          "Could not find expected separator in queue key '" + queueKey + "'");
-    }
-
-    int thirdIndex = queueKey.indexOf(KEY_SEPARATOR, secondIndex + 1);
-    if (thirdIndex == -1) {
-      throw new IllegalArgumentException(
-          "Could not find expected separator in queue key '" + queueKey + "'");
-    }
-
-    return Maps.immutableEntry(filename,
-        new ReplicationTarget(queueKey.substring(index + 1, secondIndex),
-            queueKey.substring(secondIndex + 1, thirdIndex),
-            TableId.of(queueKey.substring(thirdIndex + 1))));
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystem.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystem.java
deleted file mode 100644
index f0152dd..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystem.java
+++ /dev/null
@@ -1,50 +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.server.replication;
-
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-
-/**
- * Encapsulation of a remote system which Accumulo can replicate data to
- */
-@Deprecated
-public interface ReplicaSystem {
-
-  /**
-   * Replicate the given status to the target peer
-   *
-   * @param p Path to the resource we're reading from
-   * @param status Information to replicate
-   * @param target The peer
-   * @param helper Instance of ReplicaSystemHelper
-   * @return A new Status for the progress that was made
-   */
-  Status replicate(Path p, Status status, ReplicationTarget target, ReplicaSystemHelper helper);
-
-  /**
-   * Configure the implementation with necessary information from the system configuration
-   * <p>
-   * For example, we only need one implementation for Accumulo, but, for each peer, we have a ZK
-   * quorum and instance name
-   */
-  void configure(ServerContext context, String configuration);
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystemFactory.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystemFactory.java
deleted file mode 100644
index ed9daac..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystemFactory.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.server.replication;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.server.ServerContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Maps;
-
-@Deprecated
-public class ReplicaSystemFactory {
-  private static final Logger log = LoggerFactory.getLogger(ReplicaSystemFactory.class);
-
-  /**
-   * @param value {@link ReplicaSystem} implementation class name
-   * @return A {@link ReplicaSystem} object from the given name
-   */
-  public ReplicaSystem get(ServerContext context, String value) {
-    final Entry<String,String> entry = parseReplicaSystemConfiguration(value);
-
-    try {
-      Class<?> clz = Class.forName(entry.getKey());
-
-      if (ReplicaSystem.class.isAssignableFrom(clz)) {
-        Object o = clz.getDeclaredConstructor().newInstance();
-        ReplicaSystem rs = (ReplicaSystem) o;
-        rs.configure(context, entry.getValue());
-        return rs;
-      }
-
-      throw new IllegalArgumentException(
-          "Class is not assignable to ReplicaSystem: " + entry.getKey());
-    } catch (ReflectiveOperationException e) {
-      log.error("Error creating ReplicaSystem object", e);
-      throw new IllegalArgumentException(e);
-    }
-  }
-
-  /**
-   * Parse the configuration value for a peer into its components: {@link ReplicaSystem} class name
-   * and configuration string.
-   *
-   * @param value The configuration value for a replication peer.
-   * @return An entry where the set is the replica system name and the value is the configuration
-   *         string.
-   */
-  public Entry<String,String> parseReplicaSystemConfiguration(String value) {
-    requireNonNull(value);
-
-    int index = value.indexOf(',');
-    if (index == -1) {
-      throw new IllegalArgumentException(
-          "Expected comma separator between replication system name and configuration");
-    }
-
-    String name = value.substring(0, index);
-    String configuration = value.substring(index + 1);
-    return Maps.immutableEntry(name, configuration);
-  }
-
-  /**
-   * Generate the configuration value for a {@link ReplicaSystem} in the instance properties
-   *
-   * @param system The desired ReplicaSystem to use
-   * @param configuration Configuration string for the desired ReplicaSystem
-   * @return Value to set for peer configuration in the instance
-   */
-  public static String getPeerConfigurationValue(Class<? extends ReplicaSystem> system,
-      String configuration) {
-    String systemName = system.getName() + ",";
-    if (configuration == null) {
-      return systemName;
-    }
-
-    return systemName + configuration;
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystemHelper.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystemHelper.java
deleted file mode 100644
index e929bdc..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicaSystemHelper.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.
- */
-package org.apache.accumulo.server.replication;
-
-import static java.util.Objects.requireNonNull;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicaSystemHelper {
-  private static final Logger log = LoggerFactory.getLogger(ReplicaSystemHelper.class);
-
-  private ClientContext context;
-
-  public ReplicaSystemHelper(ClientContext context) {
-    requireNonNull(context);
-    this.context = context;
-  }
-
-  /**
-   * Record the updated Status for this file and target
-   *
-   * @param filePath Path to file being replicated
-   * @param status Updated Status after replication
-   * @param target Peer that was replicated to
-   */
-  public void recordNewStatus(Path filePath, Status status, ReplicationTarget target)
-      throws AccumuloException, TableNotFoundException {
-    try (BatchWriter bw = context.createBatchWriter(ReplicationTable.NAME)) {
-      log.debug("Recording new status for {}, {}", filePath, ProtobufUtil.toString(status));
-      Mutation m = new Mutation(filePath.toString());
-      WorkSection.add(m, target.toText(), ProtobufUtil.toValue(status));
-      bw.addMutation(m);
-    }
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
deleted file mode 100644
index b64d912..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.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.server.replication;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.conf.Property;
-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.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationUtil {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationUtil.class);
-  public static final String STATUS_FORMATTER_CLASS_NAME = StatusFormatter.class.getName();
-
-  private final ServerContext context;
-  private final ReplicaSystemFactory factory;
-
-  public ReplicationUtil(ServerContext context) {
-    this(context, new ReplicaSystemFactory());
-  }
-
-  public ReplicationUtil(ServerContext context, ReplicaSystemFactory factory) {
-    this.context = context;
-    this.factory = factory;
-  }
-
-  public int getMaxReplicationThreads(ManagerMonitorInfo mmi) {
-    int activeTservers = mmi.getTServerInfoSize();
-
-    // The number of threads each tserver will use at most to replicate data
-    int replicationThreadsPerServer =
-        context.getConfiguration().getCount(Property.REPLICATION_WORKER_THREADS);
-
-    // The total number of "slots" we have to replicate data
-    return activeTservers * replicationThreadsPerServer;
-  }
-
-  /**
-   * Extract replication peers from system configuration
-   *
-   * @return Configured replication peers
-   */
-  public Map<String,String> getPeers() {
-    Map<String,String> peers = new HashMap<>();
-
-    // Get the defined peers and what ReplicaSystem impl they're using
-    for (Entry<String,String> property : context.getConfiguration()
-        .getAllPropertiesWithPrefix(Property.REPLICATION_PEERS).entrySet()) {
-      String key = property.getKey();
-      // Filter out cruft that we don't want
-      if (!key.startsWith(Property.REPLICATION_PEER_USER.getKey())
-          && !key.startsWith(Property.REPLICATION_PEER_PASSWORD.getKey())
-          && !key.startsWith(Property.REPLICATION_PEER_KEYTAB.getKey())) {
-        String peerName = property.getKey().substring(Property.REPLICATION_PEERS.getKey().length());
-        Entry<String,String> entry;
-        try {
-          entry = factory.parseReplicaSystemConfiguration(property.getValue());
-        } catch (Exception e) {
-          log.warn("Could not instantiate ReplicaSystem for {} with configuration {}",
-              property.getKey(), property.getValue(), e);
-          continue;
-        }
-
-        peers.put(peerName, entry.getKey());
-      }
-    }
-
-    return peers;
-  }
-
-  public Set<ReplicationTarget> getReplicationTargets() {
-    // The total set of configured targets
-    final Set<ReplicationTarget> allConfiguredTargets = new HashSet<>();
-    final Map<String,TableId> tableNameToId = context.getTableNameToIdMap();
-
-    for (String table : tableNameToId.keySet()) {
-      if (MetadataTable.NAME.equals(table) || RootTable.NAME.equals(table)) {
-        continue;
-      }
-
-      TableId localId = tableNameToId.get(table);
-      if (localId == null) {
-        log.trace("Could not determine ID for {}", table);
-        continue;
-      }
-
-      TableConfiguration tableConf = context.getTableConfiguration(localId);
-      if (tableConf == null) {
-        log.trace("Could not get configuration for table {} (it no longer exists)", table);
-        continue;
-      }
-
-      for (Entry<String,String> prop : tableConf
-          .getAllPropertiesWithPrefix(Property.TABLE_REPLICATION_TARGET).entrySet()) {
-        String peerName =
-            prop.getKey().substring(Property.TABLE_REPLICATION_TARGET.getKey().length());
-        String remoteIdentifier = prop.getValue();
-        ReplicationTarget target = new ReplicationTarget(peerName, remoteIdentifier, localId);
-
-        allConfiguredTargets.add(target);
-      }
-    }
-
-    return allConfiguredTargets;
-  }
-
-  public Map<ReplicationTarget,Long> getPendingReplications() {
-    final Map<ReplicationTarget,Long> counts = new HashMap<>();
-
-    // Read over the queued work
-    BatchScanner bs;
-    try {
-      bs = context.createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4);
-    } catch (TableNotFoundException e) {
-      log.debug("No replication table exists", e);
-      return counts;
-    }
-
-    bs.setRanges(Collections.singleton(new Range()));
-    WorkSection.limit(bs);
-    try {
-      Text buffer = new Text();
-      for (Entry<Key,Value> entry : bs) {
-        Key k = entry.getKey();
-        k.getColumnQualifier(buffer);
-        ReplicationTarget target = ReplicationTarget.from(buffer);
-
-        // TODO ACCUMULO-2835 once explicit lengths are tracked, we can give size-based estimates
-        // instead of just file-based
-        Long count = counts.get(target);
-        if (count == null) {
-          counts.put(target, 1L);
-        } else {
-          counts.put(target, count + 1);
-        }
-      }
-    } finally {
-      bs.close();
-    }
-
-    return counts;
-  }
-
-  public Set<Path> getPendingReplicationPaths() {
-    final Set<Path> paths = new HashSet<>();
-
-    // Read over the queued work
-    BatchScanner bs;
-    try {
-      bs = context.createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4);
-    } catch (TableNotFoundException e) {
-      log.debug("No replication table exists", e);
-      return paths;
-    }
-
-    bs.setRanges(Collections.singleton(new Range()));
-    StatusSection.limit(bs);
-    try {
-      Text buffer = new Text();
-      for (Entry<Key,Value> entry : bs) {
-        Key k = entry.getKey();
-        k.getRow(buffer);
-        paths.add(new Path(buffer.toString()));
-      }
-    } finally {
-      bs.close();
-    }
-
-    return paths;
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
deleted file mode 100644
index c34b08d..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
+++ /dev/null
@@ -1,181 +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.server.replication;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.lexicoder.AbstractEncoder;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.TypedValueCombiner;
-import org.apache.accumulo.core.iterators.ValueFormatException;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.replication.proto.Replication.Status.Builder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Defines the rules for combining multiple {@link Status} messages
- *
- * Messages that are "closed", stay closed. "Begin" and "end" always choose the maximum of the two.
- */
-@Deprecated
-public class StatusCombiner extends TypedValueCombiner<Status> {
-  private static final Logger log = LoggerFactory.getLogger(StatusCombiner.class);
-
-  @Deprecated
-  public static class StatusEncoder extends AbstractEncoder<Status> {
-    private static final Logger log = LoggerFactory.getLogger(StatusEncoder.class);
-
-    @Override
-    public byte[] encode(Status v) {
-      return v.toByteArray();
-    }
-
-    @Override
-    public Status decode(byte[] b) {
-      // Override super because it calls decodeUnchecked, which is not as performant
-      requireNonNull(b, "cannot decode null byte array");
-      try {
-        return Status.parseFrom(b);
-      } catch (InvalidProtocolBufferException e) {
-        log.error("Failed to parse Status protocol buffer", e);
-        throw new ValueFormatException(e);
-      }
-    }
-
-    @Override
-    protected Status decodeUnchecked(byte[] b, int offset, int len) throws ValueFormatException {
-      try {
-        // have to make a copy because Status lacks the method to do this efficiently
-        byte[] boundedArr = new byte[len];
-        System.arraycopy(b, offset, boundedArr, 0, len);
-        return Status.parseFrom(boundedArr);
-      } catch (InvalidProtocolBufferException e) {
-        log.error("Failed to parse Status protocol buffer", e);
-        throw new ValueFormatException(e);
-      }
-    }
-  }
-
-  @Override
-  public IteratorOptions describeOptions() {
-    IteratorOptions io = super.describeOptions();
-    io.setName("StatusCombiner");
-    io.setDescription(
-        "Combiner that joins multiple Status protobufs to track replication metadata");
-    return io;
-  }
-
-  @Override
-  public boolean validateOptions(Map<String,String> options) {
-    return super.validateOptions(options);
-  }
-
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
-      IteratorEnvironment env) throws IOException {
-    super.init(source, options, env);
-
-    setEncoder(new StatusEncoder());
-  }
-
-  @Override
-  public Status typedReduce(Key key, Iterator<Status> iter) {
-    Builder combined = null;
-
-    while (iter.hasNext()) {
-      Status status = iter.next();
-
-      // Avoid creation of a new builder and message when we only have one
-      // message to reduce
-      if (combined == null) {
-        if (iter.hasNext()) {
-          combined = Status.newBuilder();
-        } else {
-          if (log.isTraceEnabled()) {
-            log.trace("Returned single value: {} {}", key.toStringNoTruncate(),
-                ProtobufUtil.toString(status));
-          }
-          return status;
-        }
-      }
-
-      // Add the new message in with the previous message(s)
-      combine(combined, status);
-    }
-
-    if (log.isTraceEnabled()) {
-      log.trace("Combined: {} {}", key.toStringNoTruncate(),
-          ProtobufUtil.toString(combined.build()));
-    }
-
-    return combined.build();
-  }
-
-  /**
-   * Update a {@link Builder} with another {@link Status}
-   *
-   * @param combined The Builder to combine into
-   * @param status The Status we're combining
-   */
-  public void combine(Builder combined, Status status) {
-    if (log.isTraceEnabled()) {
-      log.trace("Combining {} into {}", status.toString().replace("\n", ", "),
-          builderToString(combined));
-    }
-
-    // offset up to which replication is completed
-    combined.setBegin(Math.max(combined.getBegin(), status.getBegin()));
-
-    // offset up to which replication is necessary
-    combined.setEnd(Math.max(combined.getEnd(), status.getEnd()));
-
-    // will more data be added to the underlying file
-    combined.setClosed(combined.getClosed() | status.getClosed());
-
-    // persist the infinite end
-    combined.setInfiniteEnd(combined.getInfiniteEnd() | status.getInfiniteEnd());
-
-    // only set the createdTime if the new status has it defined
-    if (status.hasCreatedTime()) {
-      // choose the minimum (earliest) createdTime seen
-      if (combined.hasCreatedTime()) {
-        combined.setCreatedTime(Math.min(combined.getCreatedTime(), status.getCreatedTime()));
-      } else {
-        combined.setCreatedTime(status.getCreatedTime());
-      }
-    }
-  }
-
-  private String builderToString(Builder builder) {
-    return "begin: " + builder.getBegin() + ", end: " + builder.getEnd() + ", infiniteEnd: "
-        + builder.getInfiniteEnd() + ", closed: " + builder.getClosed() + ", createdTime: "
-        + builder.getCreatedTime();
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusFormatter.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusFormatter.java
deleted file mode 100644
index cb1b576..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusFormatter.java
+++ /dev/null
@@ -1,165 +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.server.replication;
-
-import java.text.DateFormat;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.accumulo.core.util.format.Formatter;
-import org.apache.accumulo.core.util.format.FormatterConfig;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Sets;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Parse and print the serialized protocol buffers used to track replication data
- */
-@Deprecated
-public class StatusFormatter implements Formatter {
-  private static final Logger log = LoggerFactory.getLogger(StatusFormatter.class);
-
-  private static final Set<Text> REPLICATION_COLFAMS =
-      Collections.unmodifiableSet(Sets.newHashSet(ReplicationSection.COLF, StatusSection.NAME,
-          WorkSection.NAME, OrderSection.NAME));
-
-  private Iterator<Entry<Key,Value>> iterator;
-  private FormatterConfig config;
-
-  /* so a new date object doesn't get created for every record in the scan result */
-  private static ThreadLocal<Date> tmpDate = new ThreadLocal<>() {
-    @Override
-    protected Date initialValue() {
-      return new Date();
-    }
-  };
-
-  @Override
-  public boolean hasNext() {
-    return iterator.hasNext();
-  }
-
-  @Override
-  public String next() {
-    Entry<Key,Value> entry = iterator.next();
-    DateFormat timestampFormat =
-        config.willPrintTimestamps() ? config.getDateFormatSupplier().get() : null;
-
-    // If we expected this to be a protobuf, try to parse it, adding a message when it fails to
-    // parse
-    if (REPLICATION_COLFAMS.contains(entry.getKey().getColumnFamily())) {
-      Status status;
-      try {
-        status = Status.parseFrom(entry.getValue().get());
-      } catch (InvalidProtocolBufferException e) {
-        log.trace("Could not deserialize protocol buffer for {}", entry.getKey(), e);
-        status = null;
-      }
-
-      return formatEntry(entry.getKey(), status, timestampFormat);
-    } else {
-      // Otherwise, we're set on a table that contains other data too (e.g. accumulo.metadata)
-      // Just do the normal thing
-      return DefaultFormatter.formatEntry(entry, timestampFormat);
-    }
-  }
-
-  public String formatEntry(Key key, Status status, DateFormat timestampFormat) {
-    StringBuilder sb = new StringBuilder();
-    Text buffer = new Text();
-
-    // append row
-    key.getRow(buffer);
-    appendText(sb, buffer).append(" ");
-
-    // append column family
-    key.getColumnFamily(buffer);
-    appendText(sb, buffer).append(":");
-
-    // append column qualifier
-    key.getColumnQualifier(buffer);
-    appendText(sb, buffer).append(" ");
-
-    // append visibility expression
-    key.getColumnVisibility(buffer);
-    sb.append(new ColumnVisibility(buffer));
-
-    // append timestamp
-    if (timestampFormat != null) {
-      tmpDate.get().setTime(key.getTimestamp());
-      sb.append(" ").append(timestampFormat.format(tmpDate.get()));
-    }
-
-    sb.append("\t");
-    // append value
-    if (status != null) {
-      sb.append(ProtobufUtil.toString(status));
-    } else {
-      sb.append("Could not deserialize Status protocol buffer");
-    }
-
-    return sb.toString();
-  }
-
-  protected StringBuilder appendText(StringBuilder sb, Text t) {
-    return appendBytes(sb, t.getBytes(), 0, t.getLength());
-  }
-
-  protected StringBuilder appendBytes(StringBuilder sb, byte[] ba, int offset, int len) {
-    for (int i = 0; i < len; i++) {
-      int c = 0xff & ba[offset + i];
-      if (c == '\\') {
-        sb.append("\\\\");
-      } else if (c >= 32 && c <= 126) {
-        sb.append((char) c);
-      } else {
-        sb.append("\\x").append(String.format("%02X", c));
-      }
-    }
-    return sb;
-  }
-
-  @Override
-  public void remove() {
-    iterator.remove();
-  }
-
-  @Override
-  public void initialize(Iterable<Entry<Key,Value>> scanner, FormatterConfig config) {
-    this.iterator = scanner.iterator();
-    this.config = new FormatterConfig(config);
-  }
-
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
deleted file mode 100644
index 7c40d0e..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
+++ /dev/null
@@ -1,224 +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.server.replication;
-
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.replication.proto.Replication.Status.Builder;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Helper methods to create Status protobuf messages
- */
-@Deprecated
-public class StatusUtil {
-
-  private static final Status INF_END_REPLICATION_STATUS, CLOSED_STATUS;
-  private static final Value INF_END_REPLICATION_STATUS_VALUE, CLOSED_STATUS_VALUE;
-
-  private static final Status.Builder CREATED_STATUS_BUILDER;
-
-  static {
-    CREATED_STATUS_BUILDER = Status.newBuilder();
-    CREATED_STATUS_BUILDER.setBegin(0);
-    CREATED_STATUS_BUILDER.setEnd(0);
-    CREATED_STATUS_BUILDER.setInfiniteEnd(false);
-    CREATED_STATUS_BUILDER.setClosed(false);
-
-    Builder builder = Status.newBuilder();
-    builder.setBegin(0);
-    builder.setEnd(0);
-    builder.setInfiniteEnd(true);
-    builder.setClosed(false);
-    INF_END_REPLICATION_STATUS = builder.build();
-    INF_END_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(INF_END_REPLICATION_STATUS);
-
-    builder = Status.newBuilder();
-    builder.setBegin(0);
-    builder.setEnd(0);
-    builder.setInfiniteEnd(true);
-    builder.setClosed(true);
-    CLOSED_STATUS = builder.build();
-    CLOSED_STATUS_VALUE = ProtobufUtil.toValue(CLOSED_STATUS);
-  }
-
-  /**
-   * Creates a {@link Status} for newly-created data that must be replicated
-   *
-   * @param recordsIngested Offset of records which need to be replicated
-   * @return A {@link Status} tracking data that must be replicated
-   */
-  public static Status ingestedUntil(long recordsIngested) {
-    return ingestedUntil(Status.newBuilder(), recordsIngested);
-  }
-
-  public static Status ingestedUntil(Builder builder, long recordsIngested) {
-    return replicatedAndIngested(builder, 0, recordsIngested);
-  }
-
-  /**
-   * @param recordsReplicated Offset of records which have been replicated
-   * @return A {@link Status} tracking data that must be replicated
-   */
-  public static Status replicated(long recordsReplicated) {
-    return replicated(Status.newBuilder(), recordsReplicated);
-  }
-
-  /**
-   * @param builder Existing {@link Builder} to use
-   * @param recordsReplicated Offset of records which have been replicated
-   * @return A {@link Status} tracking data that must be replicated
-   */
-  public static Status replicated(Status.Builder builder, long recordsReplicated) {
-    return replicatedAndIngested(builder, recordsReplicated, 0);
-  }
-
-  /**
-   * Creates a @{link Status} for a file which has new data and data which has been replicated
-   *
-   * @param recordsReplicated Offset of records which have been replicated
-   * @param recordsIngested Offset for records which need to be replicated
-   * @return A {@link Status} for the given parameters
-   */
-  public static Status replicatedAndIngested(long recordsReplicated, long recordsIngested) {
-    return replicatedAndIngested(Status.newBuilder(), recordsReplicated, recordsIngested);
-  }
-
-  /**
-   * Same as {@link #replicatedAndIngested(long, long)} but uses the provided {@link Builder}
-   *
-   * @param builder An existing builder
-   * @param recordsReplicated Offset of records which have been replicated
-   * @param recordsIngested Offset of records which need to be replicated
-   * @return A {@link Status} for the given parameters using the builder
-   */
-  public static Status replicatedAndIngested(Status.Builder builder, long recordsReplicated,
-      long recordsIngested) {
-    return builder.setBegin(recordsReplicated).setEnd(recordsIngested).setClosed(false)
-        .setInfiniteEnd(false).build();
-  }
-
-  /**
-   * @return A {@link Status} for a new file that was just created
-   */
-  public static synchronized Status fileCreated(long timeCreated) {
-    // We're using a shared builder, so we need to synchronize access on it until we make a Status
-    // (which is then immutable)
-    CREATED_STATUS_BUILDER.setCreatedTime(timeCreated);
-    return CREATED_STATUS_BUILDER.build();
-  }
-
-  /**
-   * @return A {@link Value} for a new file that was just created
-   */
-  public static Value fileCreatedValue(long timeCreated) {
-    return ProtobufUtil.toValue(fileCreated(timeCreated));
-  }
-
-  /**
-   * @return A Status representing a closed file
-   */
-  public static Status fileClosed() {
-    return CLOSED_STATUS;
-  }
-
-  /**
-   * @return A Value representing a closed file
-   */
-  public static Value fileClosedValue() {
-    return CLOSED_STATUS_VALUE;
-  }
-
-  /**
-   * @return A {@link Status} for an open file of unspecified length, all of which needs
-   *         replicating.
-   */
-  public static Status openWithUnknownLength(long timeCreated) {
-    Builder builder = Status.newBuilder();
-    builder.setBegin(0);
-    builder.setEnd(0);
-    builder.setInfiniteEnd(true);
-    builder.setClosed(false);
-    builder.setCreatedTime(timeCreated);
-    return builder.build();
-  }
-
-  /**
-   * @return A {@link Status} for an open file of unspecified length, all of which needs
-   *         replicating.
-   */
-  public static Status openWithUnknownLength() {
-    return INF_END_REPLICATION_STATUS;
-  }
-
-  /**
-   * @return A {@link Value} for an open file of unspecified length, all of which needs replicating.
-   */
-  public static Value openWithUnknownLengthValue() {
-    return INF_END_REPLICATION_STATUS_VALUE;
-  }
-
-  /**
-   * @param v Value with serialized Status
-   * @return A Status created from the Value
-   */
-  public static Status fromValue(Value v) throws InvalidProtocolBufferException {
-    return Status.parseFrom(v.get());
-  }
-
-  /**
-   * Is the given Status fully replicated and is its file ready for deletion on the source
-   *
-   * @param status a Status protobuf
-   * @return True if the file this Status references can be deleted.
-   */
-  public static boolean isSafeForRemoval(Status status) {
-    return status.getClosed() && isFullyReplicated(status);
-  }
-
-  /**
-   * Is the given Status fully replicated but potentially not yet safe for deletion
-   *
-   * @param status a Status protobuf
-   * @return True if the file this Status references is fully replicated so far
-   */
-  public static boolean isFullyReplicated(Status status) {
-    if (status.getInfiniteEnd()) {
-      return status.getBegin() == Long.MAX_VALUE;
-    } else {
-      return status.getBegin() >= status.getEnd();
-    }
-  }
-
-  /**
-   * Given the {@link Status}, is there replication work to be done
-   *
-   * @param status Status for a file
-   * @return true if replication work is required
-   */
-  public static boolean isWorkRequired(Status status) {
-    if (status.getInfiniteEnd()) {
-      return status.getBegin() != Long.MAX_VALUE;
-    } else {
-      return status.getBegin() < status.getEnd();
-    }
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java b/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java
deleted file mode 100644
index ee16b33..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java
+++ /dev/null
@@ -1,44 +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.server.replication;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-
-/**
- * Interface to allow for multiple implementations that assign replication work
- */
-@Deprecated
-public interface WorkAssigner {
-
-  /**
-   * @return The name for this WorkAssigner
-   */
-  String getName();
-
-  /**
-   * Configure the WorkAssigner implementation
-   */
-  void configure(AccumuloConfiguration conf, AccumuloClient client);
-
-  /**
-   * Assign work for replication
-   */
-  void assignWork();
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ZooKeeperInitialization.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ZooKeeperInitialization.java
deleted file mode 100644
index 41c9674..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ZooKeeperInitialization.java
+++ /dev/null
@@ -1,44 +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.server.replication;
-
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * We don't want to introduce an upgrade path to 1.7 only for some new nodes within ZooKeeper
- * <p>
- * We can take the penalty of embedding this logic into the server processes, but alleviate
- * users/developers from having to worry about the zookeeper state.
- */
-@Deprecated
-public class ZooKeeperInitialization {
-  /**
-   * Ensure that the full path to ZooKeeper nodes that will be used exist
-   */
-  public static void ensureZooKeeperInitialized(final ZooReaderWriter zooReaderWriter,
-      final String zRoot) throws KeeperException, InterruptedException {
-    // ZooReaderWriter will check existence and return if it exists, so no need to check here
-    String replicationZTServers = ReplicationConstants.ZOO_TSERVERS;
-    zooReaderWriter.mkdirs(zRoot + replicationZTServers);
-    String replicationZWorkQueue = ReplicationConstants.ZOO_WORK_QUEUE;
-    zooReaderWriter.mkdirs(zRoot + replicationZWorkQueue);
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/proto/Replication.java b/server/base/src/main/java/org/apache/accumulo/server/replication/proto/Replication.java
deleted file mode 100644
index 3a5c9eb..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/proto/Replication.java
+++ /dev/null
@@ -1,1154 +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.
- */
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: src/main/protobuf/replication.proto
-
-package org.apache.accumulo.server.replication.proto;
-
-@SuppressWarnings("unused") public final class Replication {
-  private Replication() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistryLite registry) {
-  }
-
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-    registerAllExtensions(
-        (com.google.protobuf.ExtensionRegistryLite) registry);
-  }
-  public interface StatusOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:Status)
-      com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <pre>
-     * offset where replication should start
-     * </pre>
-     *
-     * <code>optional int64 begin = 1 [default = 0];</code>
-     * @return Whether the begin field is set.
-     */
-    boolean hasBegin();
-    /**
-     * <pre>
-     * offset where replication should start
-     * </pre>
-     *
-     * <code>optional int64 begin = 1 [default = 0];</code>
-     * @return The begin.
-     */
-    long getBegin();
-
-    /**
-     * <pre>
-     * offset where data is ready for replication
-     * </pre>
-     *
-     * <code>optional int64 end = 2 [default = 0];</code>
-     * @return Whether the end field is set.
-     */
-    boolean hasEnd();
-    /**
-     * <pre>
-     * offset where data is ready for replication
-     * </pre>
-     *
-     * <code>optional int64 end = 2 [default = 0];</code>
-     * @return The end.
-     */
-    long getEnd();
-
-    /**
-     * <pre>
-     * do we have a discrete 'end'
-     * </pre>
-     *
-     * <code>optional bool infiniteEnd = 3 [default = false];</code>
-     * @return Whether the infiniteEnd field is set.
-     */
-    boolean hasInfiniteEnd();
-    /**
-     * <pre>
-     * do we have a discrete 'end'
-     * </pre>
-     *
-     * <code>optional bool infiniteEnd = 3 [default = false];</code>
-     * @return The infiniteEnd.
-     */
-    boolean getInfiniteEnd();
-
-    /**
-     * <pre>
-     * will more data be appended to the file
-     * </pre>
-     *
-     * <code>optional bool closed = 4 [default = false];</code>
-     * @return Whether the closed field is set.
-     */
-    boolean hasClosed();
-    /**
-     * <pre>
-     * will more data be appended to the file
-     * </pre>
-     *
-     * <code>optional bool closed = 4 [default = false];</code>
-     * @return The closed.
-     */
-    boolean getClosed();
-
-    /**
-     * <pre>
-     * when, in ms, was the file created?
-     * </pre>
-     *
-     * <code>optional int64 createdTime = 5 [default = 0];</code>
-     * @return Whether the createdTime field is set.
-     */
-    boolean hasCreatedTime();
-    /**
-     * <pre>
-     * when, in ms, was the file created?
-     * </pre>
-     *
-     * <code>optional int64 createdTime = 5 [default = 0];</code>
-     * @return The createdTime.
-     */
-    long getCreatedTime();
-  }
-  /**
-   * Protobuf type {@code Status}
-   */
-  public static final class Status extends
-      com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:Status)
-      StatusOrBuilder {
-  private static final long serialVersionUID = 0L;
-    // Use Status.newBuilder() to construct.
-    private Status(com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private Status() {
-    }
-
-    @java.lang.Override
-    @SuppressWarnings({"unused"})
-    protected java.lang.Object newInstance(
-        UnusedPrivateParameter unused) {
-      return new Status();
-    }
-
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private Status(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      if (extensionRegistry == null) {
-        throw new java.lang.NullPointerException();
-      }
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            case 8: {
-              bitField0_ |= 0x00000001;
-              begin_ = input.readInt64();
-              break;
-            }
-            case 16: {
-              bitField0_ |= 0x00000002;
-              end_ = input.readInt64();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              infiniteEnd_ = input.readBool();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              closed_ = input.readBool();
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000010;
-              createdTime_ = input.readInt64();
-              break;
-            }
-            default: {
-              if (!parseUnknownField(
-                  input, unknownFields, extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.accumulo.server.replication.proto.Replication.internal_static_Status_descriptor;
-    }
-
-    @java.lang.Override
-    protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.accumulo.server.replication.proto.Replication.internal_static_Status_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.accumulo.server.replication.proto.Replication.Status.class, org.apache.accumulo.server.replication.proto.Replication.Status.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int BEGIN_FIELD_NUMBER = 1;
-    private long begin_;
-    /**
-     * <pre>
-     * offset where replication should start
-     * </pre>
-     *
-     * <code>optional int64 begin = 1 [default = 0];</code>
-     * @return Whether the begin field is set.
-     */
-    @java.lang.Override
-    public boolean hasBegin() {
-      return ((bitField0_ & 0x00000001) != 0);
-    }
-    /**
-     * <pre>
-     * offset where replication should start
-     * </pre>
-     *
-     * <code>optional int64 begin = 1 [default = 0];</code>
-     * @return The begin.
-     */
-    @java.lang.Override
-    public long getBegin() {
-      return begin_;
-    }
-
-    public static final int END_FIELD_NUMBER = 2;
-    private long end_;
-    /**
-     * <pre>
-     * offset where data is ready for replication
-     * </pre>
-     *
-     * <code>optional int64 end = 2 [default = 0];</code>
-     * @return Whether the end field is set.
-     */
-    @java.lang.Override
-    public boolean hasEnd() {
-      return ((bitField0_ & 0x00000002) != 0);
-    }
-    /**
-     * <pre>
-     * offset where data is ready for replication
-     * </pre>
-     *
-     * <code>optional int64 end = 2 [default = 0];</code>
-     * @return The end.
-     */
-    @java.lang.Override
-    public long getEnd() {
-      return end_;
-    }
-
-    public static final int INFINITEEND_FIELD_NUMBER = 3;
-    private boolean infiniteEnd_;
-    /**
-     * <pre>
-     * do we have a discrete 'end'
-     * </pre>
-     *
-     * <code>optional bool infiniteEnd = 3 [default = false];</code>
-     * @return Whether the infiniteEnd field is set.
-     */
-    @java.lang.Override
-    public boolean hasInfiniteEnd() {
-      return ((bitField0_ & 0x00000004) != 0);
-    }
-    /**
-     * <pre>
-     * do we have a discrete 'end'
-     * </pre>
-     *
-     * <code>optional bool infiniteEnd = 3 [default = false];</code>
-     * @return The infiniteEnd.
-     */
-    @java.lang.Override
-    public boolean getInfiniteEnd() {
-      return infiniteEnd_;
-    }
-
-    public static final int CLOSED_FIELD_NUMBER = 4;
-    private boolean closed_;
-    /**
-     * <pre>
-     * will more data be appended to the file
-     * </pre>
-     *
-     * <code>optional bool closed = 4 [default = false];</code>
-     * @return Whether the closed field is set.
-     */
-    @java.lang.Override
-    public boolean hasClosed() {
-      return ((bitField0_ & 0x00000008) != 0);
-    }
-    /**
-     * <pre>
-     * will more data be appended to the file
-     * </pre>
-     *
-     * <code>optional bool closed = 4 [default = false];</code>
-     * @return The closed.
-     */
-    @java.lang.Override
-    public boolean getClosed() {
-      return closed_;
-    }
-
-    public static final int CREATEDTIME_FIELD_NUMBER = 5;
-    private long createdTime_;
-    /**
-     * <pre>
-     * when, in ms, was the file created?
-     * </pre>
-     *
-     * <code>optional int64 createdTime = 5 [default = 0];</code>
-     * @return Whether the createdTime field is set.
-     */
-    @java.lang.Override
-    public boolean hasCreatedTime() {
-      return ((bitField0_ & 0x00000010) != 0);
-    }
-    /**
-     * <pre>
-     * when, in ms, was the file created?
-     * </pre>
-     *
-     * <code>optional int64 createdTime = 5 [default = 0];</code>
-     * @return The createdTime.
-     */
-    @java.lang.Override
-    public long getCreatedTime() {
-      return createdTime_;
-    }
-
-    private byte memoizedIsInitialized = -1;
-    @java.lang.Override
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    @java.lang.Override
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) != 0)) {
-        output.writeInt64(1, begin_);
-      }
-      if (((bitField0_ & 0x00000002) != 0)) {
-        output.writeInt64(2, end_);
-      }
-      if (((bitField0_ & 0x00000004) != 0)) {
-        output.writeBool(3, infiniteEnd_);
-      }
-      if (((bitField0_ & 0x00000008) != 0)) {
-        output.writeBool(4, closed_);
-      }
-      if (((bitField0_ & 0x00000010) != 0)) {
-        output.writeInt64(5, createdTime_);
-      }
-      unknownFields.writeTo(output);
-    }
-
-    @java.lang.Override
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) != 0)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(1, begin_);
-      }
-      if (((bitField0_ & 0x00000002) != 0)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(2, end_);
-      }
-      if (((bitField0_ & 0x00000004) != 0)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(3, infiniteEnd_);
-      }
-      if (((bitField0_ & 0x00000008) != 0)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(4, closed_);
-      }
-      if (((bitField0_ & 0x00000010) != 0)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(5, createdTime_);
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.accumulo.server.replication.proto.Replication.Status)) {
-        return super.equals(obj);
-      }
-      org.apache.accumulo.server.replication.proto.Replication.Status other = (org.apache.accumulo.server.replication.proto.Replication.Status) obj;
-
-      if (hasBegin() != other.hasBegin()) return false;
-      if (hasBegin()) {
-        if (getBegin()
-            != other.getBegin()) return false;
-      }
-      if (hasEnd() != other.hasEnd()) return false;
-      if (hasEnd()) {
-        if (getEnd()
-            != other.getEnd()) return false;
-      }
-      if (hasInfiniteEnd() != other.hasInfiniteEnd()) return false;
-      if (hasInfiniteEnd()) {
-        if (getInfiniteEnd()
-            != other.getInfiniteEnd()) return false;
-      }
-      if (hasClosed() != other.hasClosed()) return false;
-      if (hasClosed()) {
-        if (getClosed()
-            != other.getClosed()) return false;
-      }
-      if (hasCreatedTime() != other.hasCreatedTime()) return false;
-      if (hasCreatedTime()) {
-        if (getCreatedTime()
-            != other.getCreatedTime()) return false;
-      }
-      if (!unknownFields.equals(other.unknownFields)) return false;
-      return true;
-    }
-
-    @SuppressWarnings("unchecked")
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptor().hashCode();
-      if (hasBegin()) {
-        hash = (37 * hash) + BEGIN_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
-            getBegin());
-      }
-      if (hasEnd()) {
-        hash = (37 * hash) + END_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
-            getEnd());
-      }
-      if (hasInfiniteEnd()) {
-        hash = (37 * hash) + INFINITEEND_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
-            getInfiniteEnd());
-      }
-      if (hasClosed()) {
-        hash = (37 * hash) + CLOSED_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(
-            getClosed());
-      }
-      if (hasCreatedTime()) {
-        hash = (37 * hash) + CREATEDTIME_FIELD_NUMBER;
-        hash = (53 * hash) + com.google.protobuf.Internal.hashLong(
-            getCreatedTime());
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        java.nio.ByteBuffer data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        java.nio.ByteBuffer data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.accumulo.server.replication.proto.Replication.Status parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    @java.lang.Override
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.accumulo.server.replication.proto.Replication.Status prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    @java.lang.Override
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code Status}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:Status)
-        org.apache.accumulo.server.replication.proto.Replication.StatusOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.accumulo.server.replication.proto.Replication.internal_static_Status_descriptor;
-      }
-
-      @java.lang.Override
-      protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.accumulo.server.replication.proto.Replication.internal_static_Status_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.accumulo.server.replication.proto.Replication.Status.class, org.apache.accumulo.server.replication.proto.Replication.Status.Builder.class);
-      }
-
-      // Construct using org.apache.accumulo.server.replication.proto.Replication.Status.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-        }
-      }
-      @java.lang.Override
-      public Builder clear() {
-        super.clear();
-        begin_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        end_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        infiniteEnd_ = false;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        closed_ = false;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        createdTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        return this;
-      }
-
-      @java.lang.Override
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.accumulo.server.replication.proto.Replication.internal_static_Status_descriptor;
-      }
-
-      @java.lang.Override
-      public org.apache.accumulo.server.replication.proto.Replication.Status getDefaultInstanceForType() {
-        return org.apache.accumulo.server.replication.proto.Replication.Status.getDefaultInstance();
-      }
-
-      @java.lang.Override
-      public org.apache.accumulo.server.replication.proto.Replication.Status build() {
-        org.apache.accumulo.server.replication.proto.Replication.Status result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      @java.lang.Override
-      public org.apache.accumulo.server.replication.proto.Replication.Status buildPartial() {
-        org.apache.accumulo.server.replication.proto.Replication.Status result = new org.apache.accumulo.server.replication.proto.Replication.Status(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) != 0)) {
-          result.begin_ = begin_;
-          to_bitField0_ |= 0x00000001;
-        }
-        if (((from_bitField0_ & 0x00000002) != 0)) {
-          result.end_ = end_;
-          to_bitField0_ |= 0x00000002;
-        }
-        if (((from_bitField0_ & 0x00000004) != 0)) {
-          result.infiniteEnd_ = infiniteEnd_;
-          to_bitField0_ |= 0x00000004;
-        }
-        if (((from_bitField0_ & 0x00000008) != 0)) {
-          result.closed_ = closed_;
-          to_bitField0_ |= 0x00000008;
-        }
-        if (((from_bitField0_ & 0x00000010) != 0)) {
-          result.createdTime_ = createdTime_;
-          to_bitField0_ |= 0x00000010;
-        }
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      @java.lang.Override
-      public Builder clone() {
-        return super.clone();
-      }
-      @java.lang.Override
-      public Builder setField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
-          java.lang.Object value) {
-        return super.setField(field, value);
-      }
-      @java.lang.Override
-      public Builder clearField(
-          com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return super.clearField(field);
-      }
-      @java.lang.Override
-      public Builder clearOneof(
-          com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return super.clearOneof(oneof);
-      }
-      @java.lang.Override
-      public Builder setRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, java.lang.Object value) {
-        return super.setRepeatedField(field, index, value);
-      }
-      @java.lang.Override
-      public Builder addRepeatedField(
-          com.google.protobuf.Descriptors.FieldDescriptor field,
-          java.lang.Object value) {
-        return super.addRepeatedField(field, value);
-      }
-      @java.lang.Override
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.accumulo.server.replication.proto.Replication.Status) {
-          return mergeFrom((org.apache.accumulo.server.replication.proto.Replication.Status)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.accumulo.server.replication.proto.Replication.Status other) {
-        if (other == org.apache.accumulo.server.replication.proto.Replication.Status.getDefaultInstance()) return this;
-        if (other.hasBegin()) {
-          setBegin(other.getBegin());
-        }
-        if (other.hasEnd()) {
-          setEnd(other.getEnd());
-        }
-        if (other.hasInfiniteEnd()) {
-          setInfiniteEnd(other.getInfiniteEnd());
-        }
-        if (other.hasClosed()) {
-          setClosed(other.getClosed());
-        }
-        if (other.hasCreatedTime()) {
-          setCreatedTime(other.getCreatedTime());
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      @java.lang.Override
-      public final boolean isInitialized() {
-        return true;
-      }
-
-      @java.lang.Override
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.accumulo.server.replication.proto.Replication.Status parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.accumulo.server.replication.proto.Replication.Status) e.getUnfinishedMessage();
-          throw e.unwrapIOException();
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private long begin_ ;
-      /**
-       * <pre>
-       * offset where replication should start
-       * </pre>
-       *
-       * <code>optional int64 begin = 1 [default = 0];</code>
-       * @return Whether the begin field is set.
-       */
-      @java.lang.Override
-      public boolean hasBegin() {
-        return ((bitField0_ & 0x00000001) != 0);
-      }
-      /**
-       * <pre>
-       * offset where replication should start
-       * </pre>
-       *
-       * <code>optional int64 begin = 1 [default = 0];</code>
-       * @return The begin.
-       */
-      @java.lang.Override
-      public long getBegin() {
-        return begin_;
-      }
-      /**
-       * <pre>
-       * offset where replication should start
-       * </pre>
-       *
-       * <code>optional int64 begin = 1 [default = 0];</code>
-       * @param value The begin to set.
-       * @return This builder for chaining.
-       */
-      public Builder setBegin(long value) {
-        bitField0_ |= 0x00000001;
-        begin_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * offset where replication should start
-       * </pre>
-       *
-       * <code>optional int64 begin = 1 [default = 0];</code>
-       * @return This builder for chaining.
-       */
-      public Builder clearBegin() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        begin_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      private long end_ ;
-      /**
-       * <pre>
-       * offset where data is ready for replication
-       * </pre>
-       *
-       * <code>optional int64 end = 2 [default = 0];</code>
-       * @return Whether the end field is set.
-       */
-      @java.lang.Override
-      public boolean hasEnd() {
-        return ((bitField0_ & 0x00000002) != 0);
-      }
-      /**
-       * <pre>
-       * offset where data is ready for replication
-       * </pre>
-       *
-       * <code>optional int64 end = 2 [default = 0];</code>
-       * @return The end.
-       */
-      @java.lang.Override
-      public long getEnd() {
-        return end_;
-      }
-      /**
-       * <pre>
-       * offset where data is ready for replication
-       * </pre>
-       *
-       * <code>optional int64 end = 2 [default = 0];</code>
-       * @param value The end to set.
-       * @return This builder for chaining.
-       */
-      public Builder setEnd(long value) {
-        bitField0_ |= 0x00000002;
-        end_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * offset where data is ready for replication
-       * </pre>
-       *
-       * <code>optional int64 end = 2 [default = 0];</code>
-       * @return This builder for chaining.
-       */
-      public Builder clearEnd() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        end_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      private boolean infiniteEnd_ ;
-      /**
-       * <pre>
-       * do we have a discrete 'end'
-       * </pre>
-       *
-       * <code>optional bool infiniteEnd = 3 [default = false];</code>
-       * @return Whether the infiniteEnd field is set.
-       */
-      @java.lang.Override
-      public boolean hasInfiniteEnd() {
-        return ((bitField0_ & 0x00000004) != 0);
-      }
-      /**
-       * <pre>
-       * do we have a discrete 'end'
-       * </pre>
-       *
-       * <code>optional bool infiniteEnd = 3 [default = false];</code>
-       * @return The infiniteEnd.
-       */
-      @java.lang.Override
-      public boolean getInfiniteEnd() {
-        return infiniteEnd_;
-      }
-      /**
-       * <pre>
-       * do we have a discrete 'end'
-       * </pre>
-       *
-       * <code>optional bool infiniteEnd = 3 [default = false];</code>
-       * @param value The infiniteEnd to set.
-       * @return This builder for chaining.
-       */
-      public Builder setInfiniteEnd(boolean value) {
-        bitField0_ |= 0x00000004;
-        infiniteEnd_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * do we have a discrete 'end'
-       * </pre>
-       *
-       * <code>optional bool infiniteEnd = 3 [default = false];</code>
-       * @return This builder for chaining.
-       */
-      public Builder clearInfiniteEnd() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        infiniteEnd_ = false;
-        onChanged();
-        return this;
-      }
-
-      private boolean closed_ ;
-      /**
-       * <pre>
-       * will more data be appended to the file
-       * </pre>
-       *
-       * <code>optional bool closed = 4 [default = false];</code>
-       * @return Whether the closed field is set.
-       */
-      @java.lang.Override
-      public boolean hasClosed() {
-        return ((bitField0_ & 0x00000008) != 0);
-      }
-      /**
-       * <pre>
-       * will more data be appended to the file
-       * </pre>
-       *
-       * <code>optional bool closed = 4 [default = false];</code>
-       * @return The closed.
-       */
-      @java.lang.Override
-      public boolean getClosed() {
-        return closed_;
-      }
-      /**
-       * <pre>
-       * will more data be appended to the file
-       * </pre>
-       *
-       * <code>optional bool closed = 4 [default = false];</code>
-       * @param value The closed to set.
-       * @return This builder for chaining.
-       */
-      public Builder setClosed(boolean value) {
-        bitField0_ |= 0x00000008;
-        closed_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * will more data be appended to the file
-       * </pre>
-       *
-       * <code>optional bool closed = 4 [default = false];</code>
-       * @return This builder for chaining.
-       */
-      public Builder clearClosed() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        closed_ = false;
-        onChanged();
-        return this;
-      }
-
-      private long createdTime_ ;
-      /**
-       * <pre>
-       * when, in ms, was the file created?
-       * </pre>
-       *
-       * <code>optional int64 createdTime = 5 [default = 0];</code>
-       * @return Whether the createdTime field is set.
-       */
-      @java.lang.Override
-      public boolean hasCreatedTime() {
-        return ((bitField0_ & 0x00000010) != 0);
-      }
-      /**
-       * <pre>
-       * when, in ms, was the file created?
-       * </pre>
-       *
-       * <code>optional int64 createdTime = 5 [default = 0];</code>
-       * @return The createdTime.
-       */
-      @java.lang.Override
-      public long getCreatedTime() {
-        return createdTime_;
-      }
-      /**
-       * <pre>
-       * when, in ms, was the file created?
-       * </pre>
-       *
-       * <code>optional int64 createdTime = 5 [default = 0];</code>
-       * @param value The createdTime to set.
-       * @return This builder for chaining.
-       */
-      public Builder setCreatedTime(long value) {
-        bitField0_ |= 0x00000010;
-        createdTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * when, in ms, was the file created?
-       * </pre>
-       *
-       * <code>optional int64 createdTime = 5 [default = 0];</code>
-       * @return This builder for chaining.
-       */
-      public Builder clearCreatedTime() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        createdTime_ = 0L;
-        onChanged();
-        return this;
-      }
-      @java.lang.Override
-      public final Builder setUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.setUnknownFields(unknownFields);
-      }
-
-      @java.lang.Override
-      public final Builder mergeUnknownFields(
-          final com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.mergeUnknownFields(unknownFields);
-      }
-
-
-      // @@protoc_insertion_point(builder_scope:Status)
-    }
-
-    // @@protoc_insertion_point(class_scope:Status)
-    private static final org.apache.accumulo.server.replication.proto.Replication.Status DEFAULT_INSTANCE;
-    static {
-      DEFAULT_INSTANCE = new org.apache.accumulo.server.replication.proto.Replication.Status();
-    }
-
-    public static org.apache.accumulo.server.replication.proto.Replication.Status getDefaultInstance() {
-      return DEFAULT_INSTANCE;
-    }
-
-    @java.lang.Deprecated public static final com.google.protobuf.Parser<Status>
-        PARSER = new com.google.protobuf.AbstractParser<Status>() {
-      @java.lang.Override
-      public Status parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new Status(input, extensionRegistry);
-      }
-    };
-
-    public static com.google.protobuf.Parser<Status> parser() {
-      return PARSER;
-    }
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<Status> getParserForType() {
-      return PARSER;
-    }
-
-    @java.lang.Override
-    public org.apache.accumulo.server.replication.proto.Replication.Status getDefaultInstanceForType() {
-      return DEFAULT_INSTANCE;
-    }
-
-  }
-
-  private static final com.google.protobuf.Descriptors.Descriptor
-    internal_static_Status_descriptor;
-  private static final 
-    com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_Status_fieldAccessorTable;
-
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static  com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n#src/main/protobuf/replication.proto\"u\n" +
-      "\006Status\022\020\n\005begin\030\001 \001(\003:\0010\022\016\n\003end\030\002 \001(\003:\001" +
-      "0\022\032\n\013infiniteEnd\030\003 \001(\010:\005false\022\025\n\006closed\030" +
-      "\004 \001(\010:\005false\022\026\n\013createdTime\030\005 \001(\003:\0010B0\n," +
-      "org.apache.accumulo.server.replication.p" +
-      "rotoH\001"
-    };
-    descriptor = com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
-        });
-    internal_static_Status_descriptor =
-      getDescriptor().getMessageTypes().get(0);
-    internal_static_Status_fieldAccessorTable = new
-      com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_Status_descriptor,
-        new java.lang.String[] { "Begin", "End", "InfiniteEnd", "Closed", "CreatedTime", });
-  }
-
-  // @@protoc_insertion_point(outer_class_scope)
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomNonBlockingServer.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomNonBlockingServer.java
index 929d890..7672872 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomNonBlockingServer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomNonBlockingServer.java
@@ -47,8 +47,8 @@
     try {
       selectAcceptThreadField = TNonblockingServer.class.getDeclaredField("selectAcceptThread_");
       selectAcceptThreadField.setAccessible(true);
-    } catch (Exception e) {
-      throw new RuntimeException("Failed to access required field in Thrift code.", e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException("Failed to access required field in Thrift code.", e);
     }
   }
 
@@ -80,7 +80,7 @@
       LOGGER.error("Failed to start selector thread!", e);
       return false;
     } catch (IllegalAccessException | IllegalArgumentException e) {
-      throw new RuntimeException("Exception setting customer select thread in Thrift");
+      throw new IllegalStateException("Exception setting customer select thread in Thrift");
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomThreadedSelectorServer.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomThreadedSelectorServer.java
index d807d87..06e679a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomThreadedSelectorServer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/CustomThreadedSelectorServer.java
@@ -37,7 +37,7 @@
       fbTansportField = FrameBuffer.class.getDeclaredField("trans_");
       fbTansportField.setAccessible(true);
     } catch (SecurityException | NoSuchFieldException e) {
-      throw new RuntimeException("Failed to access required field in Thrift code.", e);
+      throw new IllegalStateException("Failed to access required field in Thrift code.", e);
     }
   }
 
@@ -45,7 +45,7 @@
     try {
       return (TNonblockingTransport) fbTansportField.get(frameBuffer);
     } catch (IllegalAccessException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java
index edf5403..8b0e356 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerConnectionParams.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.server.rpc;
 
-import java.util.Properties;
-
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -32,22 +30,12 @@
  */
 public class SaslServerConnectionParams extends SaslConnectionParams {
 
-  private AuthenticationTokenSecretManager secretManager;
-
-  public SaslServerConnectionParams(AccumuloConfiguration conf, AuthenticationToken token) {
-    this(conf, token, null);
-  }
+  private final AuthenticationTokenSecretManager secretManager;
 
   public SaslServerConnectionParams(AccumuloConfiguration conf, AuthenticationToken token,
       AuthenticationTokenSecretManager secretManager) {
     super(conf, token);
-    setSecretManager(secretManager);
-  }
-
-  public SaslServerConnectionParams(Properties clientProps, AuthenticationToken token,
-      AuthenticationTokenSecretManager secretManager) {
-    super(clientProps, token);
-    setSecretManager(secretManager);
+    this.secretManager = secretManager;
   }
 
   @Override
@@ -64,8 +52,4 @@
   public AuthenticationTokenSecretManager getSecretManager() {
     return secretManager;
   }
-
-  public void setSecretManager(AuthenticationTokenSecretManager secretManager) {
-    this.secretManager = secretManager;
-  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java
index 871d452..5cd5afd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/SaslServerDigestCallbackHandler.java
@@ -45,7 +45,7 @@
   private static final Logger log = LoggerFactory.getLogger(SaslServerDigestCallbackHandler.class);
   private static final String NAME = SaslServerDigestCallbackHandler.class.getSimpleName();
 
-  private AuthenticationTokenSecretManager secretManager;
+  private final AuthenticationTokenSecretManager secretManager;
 
   public SaslServerDigestCallbackHandler(AuthenticationTokenSecretManager secretManager) {
     this.secretManager = secretManager;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/ServerAddress.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/ServerAddress.java
index 2b06387..0e033b9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/ServerAddress.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/ServerAddress.java
@@ -18,9 +18,10 @@
  */
 package org.apache.accumulo.server.rpc;
 
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.thrift.server.TServer;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Encapsulate a Thrift server and the address, host and port, to which it is bound.
  */
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/TCredentialsUpdatingInvocationHandler.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/TCredentialsUpdatingInvocationHandler.java
index 66b3ab7..f22dbbe 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/TCredentialsUpdatingInvocationHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/TCredentialsUpdatingInvocationHandler.java
@@ -60,7 +60,6 @@
   @Override
   public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
     updateArgs(args);
-
     return invokeMethod(method, args);
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
index 642d81a..2491481 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
@@ -47,7 +47,6 @@
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.UGIAssumingTransportFactory;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.ThreadPools;
@@ -74,6 +73,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 import com.google.common.primitives.Ints;
 
 /**
@@ -420,7 +420,7 @@
       socketEnabledProtocols.retainAll(Arrays.asList(protocols));
       if (socketEnabledProtocols.isEmpty()) {
         // Bad configuration...
-        throw new RuntimeException(
+        throw new IllegalStateException(
             "No available protocols available for secure socket. Available protocols: "
                 + Arrays.toString(sslServerSock.getEnabledProtocols()) + ", allowed protocols: "
                 + Arrays.toString(protocols));
@@ -511,7 +511,7 @@
           + " the Accumulo hosts files (e.g. managers, tservers) are the FQDN for"
           + " each host when using SASL.", fqdn, hostname);
       transport.close();
-      throw new RuntimeException("SASL requires that the address the thrift"
+      throw new IllegalStateException("SASL requires that the address the thrift"
           + " server listens on is the same as the FQDN for this host");
     }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java
index bb05a46..7e497c3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/ThriftProcessorTypes.java
@@ -24,11 +24,11 @@
 import org.apache.accumulo.core.gc.thrift.GCMonitorService;
 import org.apache.accumulo.core.manager.thrift.FateService;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.client.ClientServiceHandler;
@@ -81,23 +81,25 @@
   private static final ThriftProcessorTypes<ManagerClientService.Client> MANAGER =
       new ThriftProcessorTypes<>(ThriftClientTypes.MANAGER);
 
-  private static final ThriftProcessorTypes<ReplicationCoordinator.Client> REPLICATION_COORDINATOR =
-      new ThriftProcessorTypes<>(ThriftClientTypes.REPLICATION_COORDINATOR);
-
-  private static final ThriftProcessorTypes<ReplicationServicer.Client> REPLICATION_SERVICER =
-      new ThriftProcessorTypes<>(ThriftClientTypes.REPLICATION_SERVICER);
-
   @VisibleForTesting
-  public static final ThriftProcessorTypes<TabletClientService.Client> TABLET_SERVER =
+  public static final ThriftProcessorTypes<TabletServerClientService.Client> TABLET_SERVER =
       new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_SERVER);
 
   @VisibleForTesting
-  public static final ThriftProcessorTypes<TabletScanClientService.Client> TABLET_SERVER_SCAN =
+  public static final ThriftProcessorTypes<TabletScanClientService.Client> TABLET_SCAN =
       new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_SCAN);
 
-  public static TMultiplexedProcessor getCompactorTProcessor(CompactorService.Iface serviceHandler,
-      ServerContext context) {
+  public static final ThriftProcessorTypes<TabletIngestClientService.Client> TABLET_INGEST =
+      new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_INGEST);
+
+  public static final ThriftProcessorTypes<TabletManagementClientService.Client> TABLET_MGMT =
+      new ThriftProcessorTypes<>(ThriftClientTypes.TABLET_MGMT);
+
+  public static TMultiplexedProcessor getCompactorTProcessor(ClientServiceHandler clientHandler,
+      CompactorService.Iface serviceHandler, ServerContext context) {
     TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
+    muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor(
+        ClientService.Processor.class, ClientService.Iface.class, clientHandler, context));
     muxProcessor.registerProcessor(COMPACTOR.getServiceName(), COMPACTOR.getTProcessor(
         CompactorService.Processor.class, CompactorService.Iface.class, serviceHandler, context));
     return muxProcessor;
@@ -131,45 +133,37 @@
     return muxProcessor;
   }
 
-  public static TMultiplexedProcessor getReplicationCoordinatorTProcessor(
-      ReplicationCoordinator.Iface serviceHandler, ServerContext context) {
+  public static TMultiplexedProcessor getScanServerTProcessor(ClientServiceHandler clientHandler,
+      TabletScanClientService.Iface tserverHandler, ServerContext context) {
     TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
-    muxProcessor.registerProcessor(REPLICATION_COORDINATOR.getServiceName(),
-        REPLICATION_COORDINATOR.getTProcessor(ReplicationCoordinator.Processor.class,
-            ReplicationCoordinator.Iface.class, serviceHandler, context));
-    return muxProcessor;
-  }
-
-  public static TMultiplexedProcessor getReplicationClientTProcessor(
-      ReplicationServicer.Iface serviceHandler, ServerContext context) {
-    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
-    muxProcessor.registerProcessor(REPLICATION_SERVICER.getServiceName(),
-        REPLICATION_SERVICER.getTProcessor(ReplicationServicer.Processor.class,
-            ReplicationServicer.Iface.class, serviceHandler, context));
-    return muxProcessor;
-  }
-
-  public static TMultiplexedProcessor
-      getScanServerTProcessor(TabletScanClientService.Iface tserverHandler, ServerContext context) {
-    TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
-    muxProcessor.registerProcessor(TABLET_SERVER_SCAN.getServiceName(),
-        TABLET_SERVER_SCAN.getTProcessor(TabletScanClientService.Processor.class,
+    muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor(
+        ClientService.Processor.class, ClientService.Iface.class, clientHandler, context));
+    muxProcessor.registerProcessor(TABLET_SCAN.getServiceName(),
+        TABLET_SCAN.getTProcessor(TabletScanClientService.Processor.class,
             TabletScanClientService.Iface.class, tserverHandler, context));
     return muxProcessor;
   }
 
   public static TMultiplexedProcessor getTabletServerTProcessor(ClientServiceHandler clientHandler,
-      TabletClientService.Iface tserverHandler, TabletScanClientService.Iface tserverScanHandler,
-      ServerContext context) {
+      TabletServerClientService.Iface tserverHandler,
+      TabletScanClientService.Iface tserverScanHandler,
+      TabletIngestClientService.Iface tserverIngestHandler,
+      TabletManagementClientService.Iface tserverMgmtHandler, ServerContext context) {
     TMultiplexedProcessor muxProcessor = new TMultiplexedProcessor();
     muxProcessor.registerProcessor(CLIENT.getServiceName(), CLIENT.getTProcessor(
         ClientService.Processor.class, ClientService.Iface.class, clientHandler, context));
     muxProcessor.registerProcessor(TABLET_SERVER.getServiceName(),
-        TABLET_SERVER.getTProcessor(TabletClientService.Processor.class,
-            TabletClientService.Iface.class, tserverHandler, context));
-    muxProcessor.registerProcessor(TABLET_SERVER_SCAN.getServiceName(),
-        TABLET_SERVER_SCAN.getTProcessor(TabletScanClientService.Processor.class,
+        TABLET_SERVER.getTProcessor(TabletServerClientService.Processor.class,
+            TabletServerClientService.Iface.class, tserverHandler, context));
+    muxProcessor.registerProcessor(TABLET_SCAN.getServiceName(),
+        TABLET_SCAN.getTProcessor(TabletScanClientService.Processor.class,
             TabletScanClientService.Iface.class, tserverScanHandler, context));
+    muxProcessor.registerProcessor(TABLET_INGEST.getServiceName(),
+        TABLET_INGEST.getTProcessor(TabletIngestClientService.Processor.class,
+            TabletIngestClientService.Iface.class, tserverIngestHandler, context));
+    muxProcessor.registerProcessor(TABLET_MGMT.getServiceName(),
+        TABLET_MGMT.getTProcessor(TabletManagementClientService.Processor.class,
+            TabletManagementClientService.Iface.class, tserverMgmtHandler, context));
     return muxProcessor;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/TimedProcessor.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/TimedProcessor.java
index 30b7eec..4148cfb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/TimedProcessor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/TimedProcessor.java
@@ -33,7 +33,7 @@
 
   private final TProcessor other;
   private final ThriftMetrics thriftMetrics;
-  private long idleStart = 0;
+  private long idleStart;
 
   public TimedProcessor(final TProcessor next, final MetricsInfo metricsInfo) {
     this.other = next;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/UGIAssumingProcessor.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/UGIAssumingProcessor.java
index 001bf49..d89e874 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/UGIAssumingProcessor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/UGIAssumingProcessor.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.server.rpc;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 
 import javax.security.sasl.SaslServer;
 
@@ -55,7 +56,7 @@
       this.loginUser = UserGroupInformation.getLoginUser();
     } catch (IOException e) {
       log.error("Failed to obtain login user", e);
-      throw new RuntimeException("Failed to obtain login user", e);
+      throw new UncheckedIOException("Failed to obtain login user", e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
index 3718a91..01debd9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
@@ -39,7 +39,7 @@
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
 import org.apache.accumulo.core.manager.thrift.FateOperation;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -81,8 +81,8 @@
   }
 
   private boolean shouldAudit(TCredentials credentials, TableId tableId) {
-    return (audit.isInfoEnabled() || audit.isWarnEnabled()) && !tableId.equals(MetadataTable.ID)
-        && shouldAudit(credentials);
+    return (audit.isInfoEnabled() || audit.isWarnEnabled())
+        && !tableId.equals(AccumuloTable.METADATA.tableId()) && shouldAudit(credentials);
   }
 
   // Is INFO the right level to check? Do we even need that check?
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 3627765..ff2b462 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -26,7 +26,6 @@
 import java.util.Set;
 import java.util.function.Supplier;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -44,8 +43,7 @@
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.manager.thrift.FateOperation;
-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.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
@@ -106,7 +104,7 @@
   protected SecurityOperation(ServerContext context, Authorizor author, Authenticator authent,
       PermissionHandler pm) {
     this.context = context;
-    zkUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    zkUserPath = context.zkUserPath();
     zooCache = new ZooCache(context.getZooReader(), null);
     rootUserName = Suppliers.memoize(() -> new String(zooCache.get(zkUserPath), UTF_8));
     authorizor = author;
@@ -116,7 +114,7 @@
     if (!authorizor.validSecurityHandlers(authenticator, pm)
         || !authenticator.validSecurityHandlers()
         || !permHandle.validSecurityHandlers(authent, author)) {
-      throw new RuntimeException(authorizor + ", " + authenticator + ", and " + pm
+      throw new IllegalStateException(authorizor + ", " + authenticator + ", and " + pm
           + " do not play nice with each other. Please choose authentication and"
           + " authorization mechanisms that are compatible with one another.");
     }
@@ -135,11 +133,11 @@
     authorizor.initializeSecurity(credentials, rootPrincipal);
     permHandle.initializeSecurity(credentials, rootPrincipal);
     try {
-      permHandle.grantTablePermission(rootPrincipal, MetadataTable.ID.canonical(),
+      permHandle.grantTablePermission(rootPrincipal, AccumuloTable.METADATA.tableId().canonical(),
           TablePermission.ALTER_TABLE);
     } catch (TableNotFoundException e) {
       // Shouldn't happen
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -365,11 +363,8 @@
       boolean useCached) throws ThriftSecurityException {
     targetUserExists(user);
 
-    @SuppressWarnings("deprecation")
-    TableId replicationTableId = org.apache.accumulo.core.replication.ReplicationTable.ID;
-
-    if ((table.equals(MetadataTable.ID) || table.equals(RootTable.ID)
-        || table.equals(replicationTableId)) && permission.equals(TablePermission.READ)) {
+    if ((table.equals(AccumuloTable.METADATA.tableId())
+        || table.equals(AccumuloTable.ROOT.tableId())) && permission.equals(TablePermission.READ)) {
       return true;
     }
 
@@ -956,11 +951,4 @@
     return hasTablePermission(credentials, tableId, namespaceId, TablePermission.GET_SUMMARIES,
         false);
   }
-
-  public boolean validateStoredUserCreditentials() {
-    if (authenticator instanceof ZKAuthenticator) {
-      return !((ZKAuthenticator) authenticator).hasOutdatedHashes();
-    }
-    return true;
-  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityUtil.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityUtil.java
index 7d5c7f6..151d708 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityUtil.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.server.security;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.net.InetAddress;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -71,7 +72,7 @@
       }
     }
 
-    throw new RuntimeException(
+    throw new IllegalStateException(
         "Failed to perform Kerberos login for " + principal + " using  " + keyTab);
   }
 
@@ -107,7 +108,7 @@
       return org.apache.hadoop.security.SecurityUtil.getServerPrincipal(configuredPrincipal,
           InetAddress.getLocalHost().getCanonicalHostName());
     } catch (IOException e) {
-      throw new RuntimeException(
+      throw new UncheckedIOException(
           "Could not convert configured server principal: " + configuredPrincipal, e);
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java
index ddaa814..ac7d356 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java
@@ -162,7 +162,7 @@
         keyDistributor.advertise(newKey);
       } catch (KeeperException | InterruptedException e) {
         log.error("Failed to advertise AuthenticationKey in ZooKeeper. Exiting.", e);
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
 
       lastKeyUpdate = now;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java
index 52d7355..64a4fa4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java
@@ -94,7 +94,7 @@
         }
         // Ensure that the user doesn't try to extend the expiration date -- they may only limit it
         if (requestedExpirationDate > identifier.getExpirationDate()) {
-          throw new RuntimeException("Requested token lifetime exceeds configured maximum");
+          throw new IllegalStateException("Requested token lifetime exceeds configured maximum");
         }
         log.trace("Overriding token expiration date from {} to {}", identifier.getExpirationDate(),
             requestedExpirationDate);
@@ -256,7 +256,7 @@
           keyDistributor.remove(key);
         } catch (KeeperException | InterruptedException e) {
           log.error("Failed to remove AuthenticationKey from ZooKeeper. Exiting", e);
-          throw new RuntimeException(e);
+          throw new IllegalStateException(e);
         }
       }
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java
index 45104f3..7aa2306 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java
@@ -24,7 +24,6 @@
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
@@ -63,7 +62,7 @@
     zooCache = new ZooCache(context.getZooReader(), null);
     impersonation = new UserImpersonation(context.getConfiguration());
     zkAuthenticator.initialize(context);
-    zkUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    zkUserPath = context.zkUserPath();
   }
 
   @Override
@@ -103,7 +102,7 @@
       }
     } catch (KeeperException | InterruptedException e) {
       log.error("Failed to initialize security", e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -159,7 +158,7 @@
       throw new AccumuloSecurityException(principal, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("Interrupted trying to create node for user", e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
index 86fa5a0..10b7c4b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
@@ -21,12 +21,9 @@
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 
-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;
@@ -46,41 +43,14 @@
   private static final Logger log = LoggerFactory.getLogger(ZKAuthenticator.class);
 
   private ServerContext context;
-  private String ZKUserPath;
+  private String zkUserPath;
   private ZooCache zooCache;
 
   @Override
   public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReader(), null);
-    ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
-  }
-
-  /**
-   * Checks stored users and logs a warning containing the ones with outdated hashes.
-   */
-  public boolean hasOutdatedHashes() {
-    List<String> outdatedUsers = new LinkedList<>();
-    try {
-      listUsers().forEach(user -> {
-        String zpath = ZKUserPath + "/" + user;
-        byte[] zkData = zooCache.get(zpath);
-        if (ZKSecurityTool.isOutdatedPass(zkData)) {
-          outdatedUsers.add(user);
-        }
-      });
-    } catch (NullPointerException e) {
-      log.debug(
-          "initializeSecurity was not called yet, there could be no outdated passwords stored");
-    }
-    if (!outdatedUsers.isEmpty()) {
-      log.warn(
-          "Found {} user(s) with outdated password hash. These will be re-hashed"
-              + " on successful authentication. The user(s) : {}",
-          outdatedUsers.size(), String.join(", ", outdatedUsers));
-      return true;
-    }
-    return false;
+    zkUserPath = context.zkUserPath();
   }
 
   @Override
@@ -90,19 +60,19 @@
       ZooReaderWriter zoo = context.getZooReaderWriter();
       synchronized (zooCache) {
         zooCache.clear();
-        if (zoo.exists(ZKUserPath)) {
-          zoo.recursiveDelete(ZKUserPath, NodeMissingPolicy.SKIP);
-          log.info("Removed {}/ from zookeeper", ZKUserPath);
+        if (zoo.exists(zkUserPath)) {
+          zoo.recursiveDelete(zkUserPath, NodeMissingPolicy.SKIP);
+          log.info("Removed {}/ from zookeeper", zkUserPath);
         }
 
         // prep parent node of users with root username
-        zoo.putPersistentData(ZKUserPath, principal.getBytes(UTF_8), NodeExistsPolicy.FAIL);
+        zoo.putPersistentData(zkUserPath, principal.getBytes(UTF_8), NodeExistsPolicy.FAIL);
 
         constructUser(principal, ZKSecurityTool.createPass(token));
       }
     } catch (KeeperException | AccumuloException | InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -115,13 +85,13 @@
     synchronized (zooCache) {
       zooCache.clear();
       ZooReaderWriter zoo = context.getZooReaderWriter();
-      zoo.putPrivatePersistentData(ZKUserPath + "/" + user, pass, NodeExistsPolicy.FAIL);
+      zoo.putPrivatePersistentData(zkUserPath + "/" + user, pass, NodeExistsPolicy.FAIL);
     }
   }
 
   @Override
   public Set<String> listUsers() {
-    return new TreeSet<>(zooCache.getChildren(ZKUserPath));
+    return new TreeSet<>(zooCache.getChildren(zkUserPath));
   }
 
   @Override
@@ -140,7 +110,7 @@
       throw new AccumuloSecurityException(principal, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } catch (AccumuloException e) {
       log.error("{}", e.getMessage(), e);
       throw new AccumuloSecurityException(principal, SecurityErrorCode.DEFAULT_SECURITY_ERROR, e);
@@ -152,12 +122,12 @@
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        context.getZooReaderWriter().recursiveDelete(ZKUserPath + "/" + user,
+        context.getZooReaderWriter().recursiveDelete(zkUserPath + "/" + user,
             NodeMissingPolicy.FAIL);
       }
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } catch (KeeperException e) {
       if (e.code().equals(KeeperException.Code.NONODE)) {
         throw new AccumuloSecurityException(user, SecurityErrorCode.USER_DOESNT_EXIST, e);
@@ -177,8 +147,8 @@
     if (userExists(principal)) {
       try {
         synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + principal);
-          context.getZooReaderWriter().putPrivatePersistentData(ZKUserPath + "/" + principal,
+          zooCache.clear(zkUserPath + "/" + principal);
+          context.getZooReaderWriter().putPrivatePersistentData(zkUserPath + "/" + principal,
               ZKSecurityTool.createPass(pt.getPassword()), NodeExistsPolicy.OVERWRITE);
         }
       } catch (KeeperException e) {
@@ -186,7 +156,7 @@
         throw new AccumuloSecurityException(principal, SecurityErrorCode.CONNECTION_ERROR, e);
       } catch (InterruptedException e) {
         log.error("{}", e.getMessage(), e);
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       } catch (AccumuloException e) {
         log.error("{}", e.getMessage(), e);
         throw new AccumuloSecurityException(principal, SecurityErrorCode.DEFAULT_SECURITY_ERROR, e);
@@ -199,7 +169,7 @@
 
   @Override
   public boolean userExists(String user) {
-    return zooCache.get(ZKUserPath + "/" + user) != null;
+    return zooCache.get(zkUserPath + "/" + user) != null;
   }
 
   @Override
@@ -215,7 +185,7 @@
     }
     PasswordToken pt = (PasswordToken) token;
     byte[] zkData;
-    String zpath = ZKUserPath + "/" + principal;
+    String zpath = zkUserPath + "/" + principal;
     zkData = zooCache.get(zpath);
     boolean result = authenticateUser(principal, pt, zkData);
     if (!result) {
@@ -230,28 +200,7 @@
     if (zkData == null) {
       return false;
     }
-
-    // if the hash does not match the outdated format use Crypt to verify it
-    if (!ZKSecurityTool.isOutdatedPass(zkData)) {
-      return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
-    }
-
-    @SuppressWarnings("deprecation")
-    boolean oldFormatValidates = ZKSecurityTool.checkPass(pt.getPassword(), zkData);
-    if (!oldFormatValidates) {
-      // if password does not match we are done
-      return false;
-    }
-
-    // if the password is correct we have to update the stored hash with new algorithm
-    try {
-      log.debug("Upgrading hashed password for {} to new format", principal);
-      changePassword(principal, pt);
-      return true;
-    } catch (AccumuloSecurityException e) {
-      log.error("Failed to upgrade hashed password for {} to new format", principal, e);
-    }
-    return false;
+    return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
index e23b13e..8c0691d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
@@ -43,19 +43,19 @@
   private final String ZKUserAuths = "/Authorizations";
 
   private ServerContext context;
-  private String ZKUserPath;
+  private String zkUserPath;
   private ZooCache zooCache;
 
   @Override
   public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReader(), null);
-    ZKUserPath = ZKSecurityTool.getInstancePath(context.getInstanceID()) + "/users";
+    zkUserPath = context.zkUserPath();
   }
 
   @Override
   public Authorizations getCachedUserAuthorizations(String user) {
-    byte[] authsBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserAuths);
+    byte[] authsBytes = zooCache.get(zkUserPath + "/" + user + ZKUserAuths);
     if (authsBytes != null) {
       return ZKSecurityTool.convertAuthorizations(authsBytes);
     }
@@ -75,16 +75,16 @@
     // create the root user with no record-level authorizations
     try {
       // prep parent node of users with root username
-      if (!zoo.exists(ZKUserPath)) {
-        zoo.putPersistentData(ZKUserPath, rootuser.getBytes(UTF_8), NodeExistsPolicy.FAIL);
+      if (!zoo.exists(zkUserPath)) {
+        zoo.putPersistentData(zkUserPath, rootuser.getBytes(UTF_8), NodeExistsPolicy.FAIL);
       }
 
       initUser(rootuser);
-      zoo.putPersistentData(ZKUserPath + "/" + rootuser + ZKUserAuths,
+      zoo.putPersistentData(zkUserPath + "/" + rootuser + ZKUserAuths,
           ZKSecurityTool.convertAuthorizations(Authorizations.EMPTY), NodeExistsPolicy.FAIL);
     } catch (KeeperException | InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -92,13 +92,13 @@
   public void initUser(String user) throws AccumuloSecurityException {
     ZooReaderWriter zoo = context.getZooReaderWriter();
     try {
-      zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
+      zoo.putPersistentData(zkUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
     } catch (KeeperException e) {
       log.error("{}", e.getMessage(), e);
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -107,12 +107,12 @@
     try {
       synchronized (zooCache) {
         ZooReaderWriter zoo = context.getZooReaderWriter();
-        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserAuths, NodeMissingPolicy.SKIP);
-        zooCache.clear(ZKUserPath + "/" + user);
+        zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserAuths, NodeMissingPolicy.SKIP);
+        zooCache.clear(zkUserPath + "/" + user);
       }
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } catch (KeeperException e) {
       log.error("{}", e.getMessage(), e);
       if (e.code().equals(KeeperException.Code.NONODE)) {
@@ -129,7 +129,7 @@
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        context.getZooReaderWriter().putPersistentData(ZKUserPath + "/" + user + ZKUserAuths,
+        context.getZooReaderWriter().putPersistentData(zkUserPath + "/" + user + ZKUserAuths,
             ZKSecurityTool.convertAuthorizations(authorizations), NodeExistsPolicy.OVERWRITE);
       }
     } catch (KeeperException e) {
@@ -137,7 +137,7 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index ea306f1..750c7d1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@ -39,8 +39,7 @@
 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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -55,7 +54,7 @@
   private static final Logger log = LoggerFactory.getLogger(ZKPermHandler.class);
 
   private ZooReaderWriter zoo;
-  private String ZKUserPath;
+  private String zkUserPath;
   private String ZKTablePath;
   private String ZKNamespacePath;
   private ZooCache zooCache;
@@ -68,7 +67,7 @@
     zooCache = new ZooCache(context.getZooReader(), null);
     zoo = context.getZooReaderWriter();
     InstanceId instanceId = context.getInstanceID();
-    ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
+    zkUserPath = context.zkUserPath();
     ZKTablePath = ZKSecurityTool.getInstancePath(instanceId) + "/tables";
     ZKNamespacePath = ZKSecurityTool.getInstancePath(instanceId) + "/namespaces";
   }
@@ -78,7 +77,7 @@
       throws TableNotFoundException {
     byte[] serializedPerms;
     try {
-      String path = ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table;
+      String path = zkUserPath + "/" + user + ZKUserTablePerms + "/" + table;
       zoo.sync(path);
       serializedPerms = zoo.getData(path);
     } catch (KeeperException e) {
@@ -115,7 +114,7 @@
 
   @Override
   public boolean hasCachedTablePermission(String user, String table, TablePermission permission) {
-    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+    byte[] serializedPerms = zooCache.get(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table);
     if (serializedPerms != null) {
       return ZKSecurityTool.convertTablePermissions(serializedPerms).contains(permission);
     }
@@ -127,7 +126,7 @@
       NamespacePermission permission) throws NamespaceNotFoundException {
     byte[] serializedPerms;
     try {
-      String path = ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace;
+      String path = zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace;
       zoo.sync(path);
       serializedPerms = zoo.getData(path);
     } catch (KeeperException e) {
@@ -167,7 +166,7 @@
   public boolean hasCachedNamespacePermission(String user, String namespace,
       NamespacePermission permission) {
     byte[] serializedPerms =
-        zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
+        zooCache.get(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
     if (serializedPerms != null) {
       return ZKSecurityTool.convertNamespacePermissions(serializedPerms).contains(permission);
     }
@@ -178,7 +177,7 @@
   public void grantSystemPermission(String user, SystemPermission permission)
       throws AccumuloSecurityException {
     try {
-      byte[] permBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
+      byte[] permBytes = zooCache.get(zkUserPath + "/" + user + ZKUserSysPerms);
       Set<SystemPermission> perms;
       if (permBytes == null) {
         perms = new TreeSet<>();
@@ -189,7 +188,7 @@
       if (perms.add(permission)) {
         synchronized (zooCache) {
           zooCache.clear();
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms,
+          zoo.putPersistentData(zkUserPath + "/" + user + ZKUserSysPerms,
               ZKSecurityTool.convertSystemPermissions(perms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -198,7 +197,7 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -206,7 +205,7 @@
   public void grantTablePermission(String user, String table, TablePermission permission)
       throws AccumuloSecurityException {
     Set<TablePermission> tablePerms;
-    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+    byte[] serializedPerms = zooCache.get(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table);
     if (serializedPerms != null) {
       tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms);
     } else {
@@ -216,8 +215,8 @@
     try {
       if (tablePerms.add(permission)) {
         synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+          zooCache.clear(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+          zoo.putPersistentData(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               ZKSecurityTool.convertTablePermissions(tablePerms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -226,7 +225,7 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -235,7 +234,7 @@
       NamespacePermission permission) throws AccumuloSecurityException {
     Set<NamespacePermission> namespacePerms;
     byte[] serializedPerms =
-        zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
+        zooCache.get(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
     if (serializedPerms != null) {
       namespacePerms = ZKSecurityTool.convertNamespacePermissions(serializedPerms);
     } else {
@@ -245,8 +244,8 @@
     try {
       if (namespacePerms.add(permission)) {
         synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+          zooCache.clear(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
+          zoo.putPersistentData(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               ZKSecurityTool.convertNamespacePermissions(namespacePerms),
               NodeExistsPolicy.OVERWRITE);
         }
@@ -256,14 +255,14 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
   @Override
   public void revokeSystemPermission(String user, SystemPermission permission)
       throws AccumuloSecurityException {
-    byte[] sysPermBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
+    byte[] sysPermBytes = zooCache.get(zkUserPath + "/" + user + ZKUserSysPerms);
 
     // User had no system permission, nothing to revoke.
     if (sysPermBytes == null) {
@@ -276,7 +275,7 @@
       if (sysPerms.remove(permission)) {
         synchronized (zooCache) {
           zooCache.clear();
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms,
+          zoo.putPersistentData(zkUserPath + "/" + user + ZKUserSysPerms,
               ZKSecurityTool.convertSystemPermissions(sysPerms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -285,14 +284,14 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
   @Override
   public void revokeTablePermission(String user, String table, TablePermission permission)
       throws AccumuloSecurityException {
-    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
+    byte[] serializedPerms = zooCache.get(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table);
 
     // User had no table permission, nothing to revoke.
     if (serializedPerms == null) {
@@ -304,10 +303,10 @@
       if (tablePerms.remove(permission)) {
         zooCache.clear();
         if (tablePerms.isEmpty()) {
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+          zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               NodeMissingPolicy.SKIP);
         } else {
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+          zoo.putPersistentData(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               ZKSecurityTool.convertTablePermissions(tablePerms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -316,7 +315,7 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -324,7 +323,7 @@
   public void revokeNamespacePermission(String user, String namespace,
       NamespacePermission permission) throws AccumuloSecurityException {
     byte[] serializedPerms =
-        zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
+        zooCache.get(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
 
     // User had no namespace permission, nothing to revoke.
     if (serializedPerms == null) {
@@ -337,10 +336,10 @@
       if (namespacePerms.remove(permission)) {
         zooCache.clear();
         if (namespacePerms.isEmpty()) {
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+          zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               NodeMissingPolicy.SKIP);
         } else {
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+          zoo.putPersistentData(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               ZKSecurityTool.convertNamespacePermissions(namespacePerms),
               NodeExistsPolicy.OVERWRITE);
         }
@@ -350,7 +349,7 @@
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -359,8 +358,8 @@
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        for (String user : zooCache.getChildren(ZKUserPath)) {
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+        for (String user : zooCache.getChildren(zkUserPath)) {
+          zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               NodeMissingPolicy.SKIP);
         }
       }
@@ -369,7 +368,7 @@
       throw new AccumuloSecurityException("unknownUser", SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -378,8 +377,8 @@
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        for (String user : zooCache.getChildren(ZKUserPath)) {
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+        for (String user : zooCache.getChildren(zkUserPath)) {
+          zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               NodeMissingPolicy.SKIP);
         }
       }
@@ -388,7 +387,7 @@
       throw new AccumuloSecurityException("unknownUser", SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -402,8 +401,10 @@
     Collections.addAll(rootPerms, SystemPermission.values());
     Map<TableId,Set<TablePermission>> tablePerms = new HashMap<>();
     // Allow the root user to flush the system tables
-    tablePerms.put(RootTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
-    tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
+    tablePerms.put(AccumuloTable.ROOT.tableId(),
+        Collections.singleton(TablePermission.ALTER_TABLE));
+    tablePerms.put(AccumuloTable.METADATA.tableId(),
+        Collections.singleton(TablePermission.ALTER_TABLE));
     // essentially the same but on the system namespace, the ALTER_TABLE permission is now redundant
     Map<NamespaceId,Set<NamespacePermission>> namespacePerms = new HashMap<>();
     namespacePerms.put(Namespace.ACCUMULO.id(),
@@ -413,12 +414,12 @@
 
     try {
       // prep parent node of users with root username
-      if (!zoo.exists(ZKUserPath)) {
-        zoo.putPersistentData(ZKUserPath, rootuser.getBytes(UTF_8), NodeExistsPolicy.FAIL);
+      if (!zoo.exists(zkUserPath)) {
+        zoo.putPersistentData(zkUserPath, rootuser.getBytes(UTF_8), NodeExistsPolicy.FAIL);
       }
 
       initUser(rootuser);
-      zoo.putPersistentData(ZKUserPath + "/" + rootuser + ZKUserSysPerms,
+      zoo.putPersistentData(zkUserPath + "/" + rootuser + ZKUserSysPerms,
           ZKSecurityTool.convertSystemPermissions(rootPerms), NodeExistsPolicy.FAIL);
       for (Entry<TableId,Set<TablePermission>> entry : tablePerms.entrySet()) {
         createTablePerm(rootuser, entry.getKey(), entry.getValue());
@@ -428,24 +429,24 @@
       }
     } catch (KeeperException | InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
   @Override
   public void initUser(String user) throws AccumuloSecurityException {
     try {
-      zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
-      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms, new byte[0],
+      zoo.putPersistentData(zkUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
+      zoo.putPersistentData(zkUserPath + "/" + user + ZKUserTablePerms, new byte[0],
           NodeExistsPolicy.SKIP);
-      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms, new byte[0],
+      zoo.putPersistentData(zkUserPath + "/" + user + ZKUserNamespacePerms, new byte[0],
           NodeExistsPolicy.SKIP);
     } catch (KeeperException e) {
       log.error("{}", e.getMessage(), e);
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -457,7 +458,7 @@
       throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
-      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+      zoo.putPersistentData(zkUserPath + "/" + user + ZKUserTablePerms + "/" + table,
           ZKSecurityTool.convertTablePermissions(perms), NodeExistsPolicy.FAIL);
     }
   }
@@ -470,7 +471,7 @@
       Set<NamespacePermission> perms) throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
-      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+      zoo.putPersistentData(zkUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
           ZKSecurityTool.convertNamespacePermissions(perms), NodeExistsPolicy.FAIL);
     }
   }
@@ -479,14 +480,14 @@
   public void cleanUser(String user) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
-        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserSysPerms, NodeMissingPolicy.SKIP);
-        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms, NodeMissingPolicy.SKIP);
-        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms, NodeMissingPolicy.SKIP);
-        zooCache.clear(ZKUserPath + "/" + user);
+        zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserSysPerms, NodeMissingPolicy.SKIP);
+        zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserTablePerms, NodeMissingPolicy.SKIP);
+        zoo.recursiveDelete(zkUserPath + "/" + user + ZKUserNamespacePerms, NodeMissingPolicy.SKIP);
+        zooCache.clear(zkUserPath + "/" + user);
       }
     } catch (InterruptedException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } catch (KeeperException e) {
       log.error("{}", e.getMessage(), e);
       if (e.code().equals(KeeperException.Code.NONODE)) {
@@ -501,7 +502,7 @@
   public boolean hasSystemPermission(String user, SystemPermission permission) {
     byte[] perms;
     try {
-      String path = ZKUserPath + "/" + user + ZKUserSysPerms;
+      String path = zkUserPath + "/" + user + ZKUserSysPerms;
       zoo.sync(path);
       perms = zoo.getData(path);
     } catch (KeeperException e) {
@@ -523,7 +524,7 @@
 
   @Override
   public boolean hasCachedSystemPermission(String user, SystemPermission permission) {
-    byte[] perms = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
+    byte[] perms = zooCache.get(zkUserPath + "/" + user + ZKUserSysPerms);
     if (perms == null) {
       return false;
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
index f28556a..af554f0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
@@ -25,9 +25,8 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.HashSet;
 import java.util.Set;
@@ -57,64 +56,6 @@
  */
 class ZKSecurityTool {
   private static final Logger log = LoggerFactory.getLogger(ZKSecurityTool.class);
-  private static final int SALT_LENGTH = 8;
-  private static final SecureRandom random = new SecureRandom();
-
-  // Generates a byte array salt of length SALT_LENGTH
-  private static byte[] generateSalt() {
-    byte[] salt = new byte[SALT_LENGTH];
-    random.nextBytes(salt);
-    return salt;
-  }
-
-  // only present for testing DO NOT USE!
-  @Deprecated(since = "2.1.0")
-  static byte[] createOutdatedPass(byte[] password) throws AccumuloException {
-    byte[] salt = generateSalt();
-    try {
-      return convertPass(password, salt);
-    } catch (NoSuchAlgorithmException e) {
-      log.error("Count not create hashed password", e);
-      throw new AccumuloException("Count not create hashed password", e);
-    }
-  }
-
-  private static final String PW_HASH_ALGORITHM_OUTDATED = "SHA-256";
-
-  private static byte[] hash(byte[] raw) throws NoSuchAlgorithmException {
-    MessageDigest md = MessageDigest.getInstance(PW_HASH_ALGORITHM_OUTDATED);
-    md.update(raw);
-    return md.digest();
-  }
-
-  @Deprecated(since = "2.1.0")
-  static boolean checkPass(byte[] password, byte[] zkData) {
-    if (zkData == null) {
-      return false;
-    }
-
-    byte[] salt = new byte[SALT_LENGTH];
-    System.arraycopy(zkData, 0, salt, 0, SALT_LENGTH);
-    byte[] passwordToCheck;
-    try {
-      passwordToCheck = convertPass(password, salt);
-    } catch (NoSuchAlgorithmException e) {
-      log.error("Count not create hashed password", e);
-      return false;
-    }
-    return MessageDigest.isEqual(passwordToCheck, zkData);
-  }
-
-  private static byte[] convertPass(byte[] password, byte[] salt) throws NoSuchAlgorithmException {
-    byte[] plainSalt = new byte[password.length + SALT_LENGTH];
-    System.arraycopy(password, 0, plainSalt, 0, password.length);
-    System.arraycopy(salt, 0, plainSalt, password.length, SALT_LENGTH);
-    byte[] hashed = hash(plainSalt);
-    byte[] saltedHash = new byte[SALT_LENGTH + hashed.length];
-    System.arraycopy(salt, 0, saltedHash, 0, SALT_LENGTH);
-    System.arraycopy(hashed, 0, saltedHash, SALT_LENGTH, hashed.length);
-    return saltedHash; // contains salt+hash(password+salt)
-  }
 
   public static byte[] createPass(byte[] password) throws AccumuloException {
     // we rely on default algorithm and salt length (SHA-512 and 8 bytes)
@@ -183,8 +124,8 @@
       }
     } catch (IOException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e); // this is impossible with ByteArrayOutputStream; crash hard if
-                                     // this happens
+      // this is impossible with ByteArrayOutputStream; crash hard if this happens
+      throw new UncheckedIOException(e);
     }
     return bytes.toByteArray();
   }
@@ -213,8 +154,8 @@
       }
     } catch (IOException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e); // this is impossible with ByteArrayOutputStream; crash hard if
-                                     // this happens
+      // this is impossible with ByteArrayOutputStream; crash hard if this happens
+      throw new UncheckedIOException(e);
     }
     return bytes.toByteArray();
   }
@@ -236,8 +177,8 @@
       }
     } catch (IOException e) {
       log.error("{}", e.getMessage(), e);
-      throw new RuntimeException(e); // this is impossible with ByteArrayOutputStream; crash hard if
-                                     // this happens
+      // this is impossible with ByteArrayOutputStream; crash hard if this happens
+      throw new UncheckedIOException(e);
     }
     return bytes.toByteArray();
   }
@@ -253,8 +194,4 @@
   public static String getInstancePath(InstanceId instanceId) {
     return Constants.ZROOT + "/" + instanceId;
   }
-
-  public static boolean isOutdatedPass(byte[] zkData) {
-    return zkData.length == 40;
-  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
index 29df98f..380942b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
@@ -178,7 +178,7 @@
       });
     } catch (Exception e) {
       log.error("FATAL Failed to transition table to state {}", newState);
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
index 45d3870..7c997d3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
@@ -53,7 +53,6 @@
     } else if (metadataTime.getType().equals(TimeType.MILLIS)) {
       return new MillisTime(metadataTime.getTime());
     } else {
-      // this should really never happen here
       throw new IllegalArgumentException("Time type unknown : " + metadataTime);
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java
index cd63313..1402612 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java
@@ -19,8 +19,7 @@
 package org.apache.accumulo.server.tablets;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.security.SecureRandom;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
@@ -39,7 +38,6 @@
 public class UniqueNameAllocator {
 
   private static final Logger log = LoggerFactory.getLogger(UniqueNameAllocator.class);
-  private static final SecureRandom random = new SecureRandom();
   private static final Property MIN_PROP = Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MIN;
   private static final Property MAX_PROP = Property.GENERAL_FILE_NAME_ALLOCATION_BATCH_SIZE_MAX;
   private static final int DEFAULT_MIN = DefaultConfiguration.getInstance().getCount(MIN_PROP);
@@ -91,7 +89,7 @@
       maxAllocation = minAllocation;
     }
 
-    int actualBatchSize = minAllocation + random.nextInt((maxAllocation - minAllocation) + 1);
+    int actualBatchSize = minAllocation + RANDOM.get().nextInt((maxAllocation - minAllocation) + 1);
     log.debug("Allocating {} filenames", actualBatchSize);
     return actualBatchSize;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index ff9c4d9..e5aadc8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -57,11 +57,11 @@
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.ReadOnlyTStore;
 import org.apache.accumulo.core.fate.ZooStore;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.manager.thrift.FateService;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
@@ -72,7 +72,6 @@
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 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.tables.TableMap;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.cli.ServerUtilOpts;
@@ -89,6 +88,7 @@
 import com.google.auto.service.AutoService;
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.Lists;
+import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -127,10 +127,6 @@
   @Parameters(commandDescription = "stop the manager")
   static class StopManagerCommand {}
 
-  @Deprecated(since = "2.1.0")
-  @Parameters(commandDescription = "stop the master (DEPRECATED -- use stopManager instead)")
-  static class StopMasterCommand {}
-
   @Parameters(commandDescription = "stop all tablet servers and the manager")
   static class StopAllCommand {}
 
@@ -167,17 +163,6 @@
     boolean users = false;
   }
 
-  private static final String RV_DEPRECATION_MSG =
-      "Randomizing tablet directories is deprecated and now does nothing. Accumulo now always"
-          + " calls the volume chooser for each file created by a tablet, so its no longer "
-          + "necessary.";
-
-  @Parameters(commandDescription = RV_DEPRECATION_MSG)
-  static class RandomizeVolumesCommand {
-    @Parameter(names = {"-t"}, description = "table to update", required = true)
-    String tableName = null;
-  }
-
   @Parameters(commandDescription = "Verify all Tablets are assigned to tablet servers")
   static class VerifyTabletAssignmentsCommand {
     @Parameter(names = {"-v", "--verbose"},
@@ -310,9 +295,6 @@
     RestoreZooCommand restoreZooOpts = new RestoreZooCommand();
     cl.addCommand("restoreZoo", restoreZooOpts);
 
-    RandomizeVolumesCommand randomizeVolumesOpts = new RandomizeVolumesCommand();
-    cl.addCommand("randomizeVolumes", randomizeVolumesOpts);
-
     StopCommand stopOpts = new StopCommand();
     cl.addCommand("stop", stopOpts);
 
@@ -322,9 +304,6 @@
     StopManagerCommand stopManagerOpts = new StopManagerCommand();
     cl.addCommand("stopManager", stopManagerOpts);
 
-    StopMasterCommand stopMasterOpts = new StopMasterCommand();
-    cl.addCommand("stopMaster", stopMasterOpts);
-
     VerifyTabletAssignmentsCommand verifyTabletAssignmentsOpts =
         new VerifyTabletAssignmentsCommand();
     cl.addCommand("verifyTabletAssigns", verifyTabletAssignmentsOpts);
@@ -382,8 +361,6 @@
         printConfig(context, dumpConfigCommand);
       } else if (cl.getParsedCommand().equals("volumes")) {
         ListVolumesUsed.listVolumes(context);
-      } else if (cl.getParsedCommand().equals("randomizeVolumes")) {
-        System.out.println(RV_DEPRECATION_MSG);
       } else if (cl.getParsedCommand().equals("verifyTabletAssigns")) {
         VerifyTabletAssignments.execute(opts.getClientProps(), verifyTabletAssignmentsOpts.verbose);
       } else if (cl.getParsedCommand().equals("changeSecret")) {
@@ -461,7 +438,7 @@
       try {
         Set<String> tables = context.tableOperations().tableIdMap().keySet();
         for (String table : tables) {
-          if (table.equals(MetadataTable.NAME)) {
+          if (table.equals(AccumuloTable.METADATA.tableName())) {
             continue;
           }
           try {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
index 2f183d4..094167e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
@@ -93,7 +93,7 @@
         recurse(zoo, root + "/" + child, v);
       }
     } catch (Exception ex) {
-      throw new RuntimeException(ex);
+      throw new IllegalStateException(ex);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
index 689ef46..1aafbec 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
@@ -32,8 +32,7 @@
 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.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.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -182,11 +181,11 @@
     Span span = TraceUtil.startSpan(CheckForMetadataProblems.class, "main");
     try (Scope scope = span.makeCurrent()) {
 
-      checkMetadataAndRootTableEntries(RootTable.NAME, opts);
+      checkMetadataAndRootTableEntries(AccumuloTable.ROOT.tableName(), opts);
       System.out.println();
-      checkMetadataAndRootTableEntries(MetadataTable.NAME, opts);
+      checkMetadataAndRootTableEntries(AccumuloTable.METADATA.tableName(), opts);
       if (sawProblems) {
-        throw new RuntimeException();
+        throw new IllegalStateException();
       }
     } catch (Exception e) {
       TraceUtil.setException(span, e, true);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ConvertConfig.java b/server/base/src/main/java/org/apache/accumulo/server/util/ConvertConfig.java
deleted file mode 100644
index 74b4be6..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ConvertConfig.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.server.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.Map;
-import java.util.stream.StreamSupport;
-
-import org.apache.accumulo.core.cli.Help;
-import org.apache.accumulo.start.spi.KeywordExecutable;
-import org.apache.hadoop.conf.Configuration;
-
-import com.beust.jcommander.Parameter;
-import com.google.auto.service.AutoService;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@AutoService(KeywordExecutable.class)
-public class ConvertConfig implements KeywordExecutable {
-
-  @Override
-  public String keyword() {
-    return "convert-config";
-  }
-
-  @Override
-  public UsageGroup usageGroup() {
-    return UsageGroup.OTHER;
-  }
-
-  @Override
-  public String description() {
-    return "Convert Accumulo configuration from XML to properties";
-  }
-
-  static class Opts extends Help {
-
-    @Parameter(names = {"-x", "-xml", "--xml"},
-        description = "Path of accumulo-site.xml to convert from")
-    String xmlPath = "./accumulo-site.xml";
-
-    @Parameter(names = {"-p", "-props", "--props"},
-        description = "Path to create new accumulo.properties")
-    String propsPath = "./accumulo.properties";
-  }
-
-  private static void writeLine(BufferedWriter w, String value) {
-    try {
-      w.write(value + "\n");
-    } catch (IOException e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "code runs in same security context as user who provided input")
-  @Override
-  public void execute(String[] args) throws Exception {
-    Opts opts = new Opts();
-    opts.parseArgs("accumulo convert-config", args);
-
-    File xmlFile = new File(opts.xmlPath);
-    if (!xmlFile.exists()) {
-      throw new IllegalArgumentException("xml config file does not exist at " + opts.xmlPath);
-    }
-
-    Path propsPath = Paths.get(opts.propsPath);
-    if (propsPath.toFile().exists()) {
-      throw new IllegalArgumentException("properties file already exists at " + opts.propsPath);
-    }
-
-    Configuration xmlConfig = new Configuration(false);
-    xmlConfig.addResource(xmlFile.toURI().toURL());
-
-    try (BufferedWriter w = Files.newBufferedWriter(propsPath, UTF_8)) {
-      StreamSupport.stream(xmlConfig.spliterator(), false).sorted(Map.Entry.comparingByKey())
-          .forEach(e -> writeLine(w, e.toString()));
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    new ConvertConfig().execute(args);
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
index 3f0bcc1..dfb6940 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ECAdmin.java
@@ -27,7 +27,6 @@
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.compaction.RunningCompaction;
 import org.apache.accumulo.core.util.compaction.RunningCompactionInfo;
@@ -41,6 +40,7 @@
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
 import com.google.auto.service.AutoService;
+import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -138,7 +138,7 @@
       coordinatorClient.cancel(TraceUtil.traceInfo(), context.rpcCreds(), ecid);
       System.out.println("Cancel sent to coordinator for " + ecid);
     } catch (Exception e) {
-      throw new RuntimeException("Exception calling cancel compaction for " + ecid, e);
+      throw new IllegalStateException("Exception calling cancel compaction for " + ecid, e);
     } finally {
       ThriftUtil.returnClient(coordinatorClient, context);
     }
@@ -189,7 +189,7 @@
         }
       });
     } catch (Exception e) {
-      throw new RuntimeException("Unable to get running compactions.", e);
+      throw new IllegalStateException("Unable to get running compactions.", e);
     } finally {
       ThriftUtil.returnClient(coordinatorClient, context);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java b/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java
deleted file mode 100644
index 9ca6389..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FileSystemMonitor.java
+++ /dev/null
@@ -1,158 +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.server.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.core.util.threads.Threads;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-public class FileSystemMonitor {
-  private static final String PROC_MOUNTS = "/proc/mounts";
-  private static final Logger log = LoggerFactory.getLogger(FileSystemMonitor.class);
-  private static final String DEVICE_PREFIX = "/dev/";
-  private static final Set<String> ACCEPTED_FILESYSTEMS = Set.of("ext3", "ext4", "xfs");
-
-  static class Mount {
-    String device;
-    String mountPoint;
-    String filesystemType;
-    Set<String> options;
-
-    Mount(String line) {
-      String[] tokens = line.split("\\s+");
-
-      device = tokens[0].trim();
-      mountPoint = tokens[1].trim();
-      filesystemType = tokens[2].trim().toLowerCase();
-
-      options = Set.of(tokens[3].split(","));
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder(32);
-      sb.append(device).append(" on ").append(mountPoint).append(" as ").append(filesystemType)
-          .append(" with options ").append(options);
-      return sb.toString();
-    }
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "procFile path not from user input")
-  static List<Mount> parse(String procFile) throws IOException {
-
-    FileReader fr = new FileReader(procFile, UTF_8);
-
-    try (BufferedReader br = new BufferedReader(fr)) {
-      return getMountsFromFile(br);
-    }
-  }
-
-  static List<Mount> getMountsFromFile(BufferedReader br) throws IOException {
-    List<Mount> mounts = new ArrayList<>();
-    String line;
-    while ((line = br.readLine()) != null) {
-      Mount mount = new Mount(line);
-      if (mount.device.startsWith(DEVICE_PREFIX)
-          && ACCEPTED_FILESYSTEMS.contains(mount.filesystemType)) {
-        log.trace("Retaining mount to check: '{}'", mount);
-        mounts.add(mount);
-      } else {
-        log.trace("Ignoring mount to check: '{}'", mount);
-      }
-    }
-    return mounts;
-  }
-
-  private Map<String,Boolean> readWriteFilesystems = new HashMap<>();
-
-  public FileSystemMonitor(final String procFile, long period, AccumuloConfiguration conf)
-      throws IOException {
-    List<Mount> mounts = parse(procFile);
-
-    // Populate readWriteFilesystems with the initial state of the mounts
-    for (Mount mount : mounts) {
-      if (mount.options.contains("rw")) {
-        readWriteFilesystems.put(mount.mountPoint, true);
-      } else if (mount.options.contains("ro")) {
-        readWriteFilesystems.put(mount.mountPoint, false);
-      } else {
-        throw new IOException("Filesystem " + mount + " does not have ro or rw option");
-      }
-    }
-
-    // Create a task to check each mount periodically to see if its state has changed.
-    for (Mount mount : mounts) {
-      ThreadPools.watchCriticalFixedDelay(conf, period,
-          Threads.createNamedRunnable(mount.mountPoint + "filesystem monitor", () -> {
-            try {
-              checkMount(mount);
-            } catch (final Exception e) {
-              Halt.halt(-42,
-                  () -> log.error("Exception while checking mount points, halting process", e));
-            }
-          }));
-    }
-
-  }
-
-  protected void checkMount(Mount mount) throws Exception {
-    if (!readWriteFilesystems.containsKey(mount.mountPoint)) {
-      if (mount.options.contains("rw")) {
-        readWriteFilesystems.put(mount.mountPoint, true);
-      } else if (mount.options.contains("ro")) {
-        readWriteFilesystems.put(mount.mountPoint, false);
-      } else {
-        throw new Exception("Filesystem " + mount + " does not have ro or rw option");
-      }
-    } else if (mount.options.contains("ro") && readWriteFilesystems.get(mount.mountPoint)) {
-      throw new Exception("Filesystem " + mount.mountPoint + " switched to read only");
-    }
-  }
-
-  public static void start(AccumuloConfiguration conf) {
-    if (new File(PROC_MOUNTS).exists()) {
-      try {
-        new FileSystemMonitor(PROC_MOUNTS, 60000, conf);
-        log.info("Filesystem monitor started");
-      } catch (IOException e) {
-        log.error("Failed to initialize file system monitor", e);
-      }
-    } else {
-      log.info("Not monitoring filesystems, " + PROC_MOUNTS + " does not exists");
-    }
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
index 3c18964..dd1dd91 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
@@ -18,9 +18,10 @@
  */
 package org.apache.accumulo.server.util;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -32,6 +33,7 @@
 import java.util.TreeMap;
 import java.util.stream.Collectors;
 
+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.PartialKey;
@@ -44,36 +46,35 @@
 import org.apache.accumulo.core.file.rfile.RFileOperations;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class FileUtil {
 
-  private static final SecureRandom random = new SecureRandom();
-
   public static class FileInfo {
-    Key firstKey = new Key();
-    Key lastKey = new Key();
+    final Text firstRow;
+    final Text lastRow;
 
-    public FileInfo(Key firstKey, Key lastKey) {
-      this.firstKey = firstKey;
-      this.lastKey = lastKey;
+    public FileInfo(Text firstRow, Text lastRow) {
+      this.firstRow = firstRow;
+      this.lastRow = lastRow;
     }
 
     public Text getFirstRow() {
-      return firstKey.getRow();
+      return firstRow;
     }
 
     public Text getLastRow() {
-      return lastKey.getRow();
+      return lastRow;
     }
   }
 
@@ -87,7 +88,7 @@
     Path result = null;
     while (result == null) {
       result = new Path(tabletDirectory + Path.SEPARATOR + "tmp/idxReduce_"
-          + String.format("%09d", random.nextInt(Integer.MAX_VALUE)));
+          + String.format("%09d", RANDOM.get().nextInt(Integer.MAX_VALUE)));
       try {
         fs.getFileStatus(result);
         result = null;
@@ -110,45 +111,47 @@
     return result;
   }
 
-  public static Collection<String> reduceFiles(ServerContext context, TableConfiguration tableConf,
-      Text prevEndRow, Text endRow, Collection<String> mapFiles, int maxFiles, Path tmpDir,
-      int pass) throws IOException {
+  public static Collection<? extends TabletFile> reduceFiles(ServerContext context,
+      TableConfiguration tableConf, Text prevEndRow, Text endRow,
+      Collection<? extends TabletFile> dataFiles, int maxFiles, Path tmpDir, int pass)
+      throws IOException {
 
-    ArrayList<String> paths = new ArrayList<>(mapFiles);
+    ArrayList<? extends TabletFile> files = new ArrayList<>(dataFiles);
 
-    if (paths.size() <= maxFiles) {
-      return paths;
+    if (files.size() <= maxFiles) {
+      return files;
     }
 
     String newDir = String.format("%s/pass_%04d", tmpDir, pass);
 
     int start = 0;
 
-    ArrayList<String> outFiles = new ArrayList<>();
+    ArrayList<UnreferencedTabletFile> outFiles = new ArrayList<>();
 
     int count = 0;
 
-    while (start < paths.size()) {
-      int end = Math.min(maxFiles + start, paths.size());
-      List<String> inFiles = paths.subList(start, end);
+    while (start < files.size()) {
+      int end = Math.min(maxFiles + start, files.size());
+      List<? extends TabletFile> inFiles = files.subList(start, end);
 
       start = end;
 
       // temporary tablet file does not conform to typical path verified in TabletFile
-      String newMapFile = String.format("%s/%04d.%s", newDir, count++, RFile.EXTENSION);
+      Path newPath = new Path(String.format("%s/%04d.%s", newDir, count++, RFile.EXTENSION));
+      FileSystem ns = context.getVolumeManager().getFileSystemByPath(newPath);
+      UnreferencedTabletFile newDataFile = UnreferencedTabletFile.of(ns, newPath);
 
-      outFiles.add(newMapFile);
-      FileSystem ns = context.getVolumeManager().getFileSystemByPath(new Path(newMapFile));
+      outFiles.add(newDataFile);
       FileSKVWriter writer = new RFileOperations().newWriterBuilder()
-          .forFile(newMapFile, ns, ns.getConf(), tableConf.getCryptoService())
+          .forFile(newDataFile, ns, ns.getConf(), tableConf.getCryptoService())
           .withTableConfiguration(tableConf).build();
       writer.startDefaultLocalityGroup();
       List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(inFiles.size());
 
       FileSKVIterator reader = null;
       try {
-        for (String file : inFiles) {
-          ns = context.getVolumeManager().getFileSystemByPath(new Path(file));
+        for (TabletFile file : inFiles) {
+          ns = context.getVolumeManager().getFileSystemByPath(file.getPath());
           reader = FileOperations.getInstance().newIndexReaderBuilder()
               .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
               .withTableConfiguration(tableConf).build();
@@ -207,25 +210,24 @@
   }
 
   public static double estimatePercentageLTE(ServerContext context, TableConfiguration tableConf,
-      String tabletDir, Text prevEndRow, Text endRow, Collection<String> mapFiles, Text splitRow)
-      throws IOException {
+      String tabletDir, Text prevEndRow, Text endRow, Collection<? extends TabletFile> dataFiles,
+      Text splitRow) throws IOException {
 
     Path tmpDir = null;
 
-    int maxToOpen =
-        context.getConfiguration().getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
-    ArrayList<FileSKVIterator> readers = new ArrayList<>(mapFiles.size());
+    int maxToOpen = getMaxFilesToOpen(context.getConfiguration());
+    ArrayList<FileSKVIterator> readers = new ArrayList<>(dataFiles.size());
 
     try {
-      if (mapFiles.size() > maxToOpen) {
+      if (dataFiles.size() > maxToOpen) {
         tmpDir = createTmpDir(context, tabletDir);
 
         log.debug("Too many indexes ({}) to open at once for {} {}, reducing in tmpDir = {}",
-            mapFiles.size(), endRow, prevEndRow, tmpDir);
+            dataFiles.size(), endRow, prevEndRow, tmpDir);
 
         long t1 = System.currentTimeMillis();
-        mapFiles =
-            reduceFiles(context, tableConf, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir, 0);
+        dataFiles =
+            reduceFiles(context, tableConf, prevEndRow, endRow, dataFiles, maxToOpen, tmpDir, 0);
         long t2 = System.currentTimeMillis();
 
         log.debug("Finished reducing indexes for {} {} in {}", endRow, prevEndRow,
@@ -237,7 +239,7 @@
       }
 
       long numKeys =
-          countIndexEntries(context, tableConf, prevEndRow, endRow, mapFiles, true, readers);
+          countIndexEntries(context, tableConf, prevEndRow, endRow, dataFiles, true, readers);
 
       if (numKeys == 0) {
         // not enough info in the index to answer the question, so instead of going to
@@ -262,8 +264,8 @@
 
       if (numLte > numKeys) {
         // something went wrong
-        throw new RuntimeException("numLte > numKeys " + numLte + " " + numKeys + " " + prevEndRow
-            + " " + endRow + " " + splitRow + " " + mapFiles);
+        throw new IllegalStateException("numLte > numKeys " + numLte + " " + numKeys + " "
+            + prevEndRow + " " + endRow + " " + splitRow + " " + dataFiles);
       }
 
       // do not want to return 0% or 100%, so add 1 and 2 below
@@ -274,41 +276,47 @@
     }
   }
 
+  @SuppressWarnings("deprecation")
+  private static int getMaxFilesToOpen(AccumuloConfiguration conf) {
+    return conf.getCount(
+        conf.resolve(Property.SPLIT_MAXOPEN, Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN));
+  }
+
   /**
    *
-   * @param mapFiles - list MapFiles to find the mid point key
+   * @param dataFiles - list of data files to find the mid point key
    *
-   *        ISSUES : This method used the index files to find the mid point. If the map files have
+   *        ISSUES : This method used the index files to find the mid point. If the data files have
    *        different index intervals this method will not return an accurate mid point. Also, it
    *        would be tricky to use this method in conjunction with an in memory map because the
    *        indexing interval is unknown.
    */
   public static SortedMap<Double,Key> findMidPoint(ServerContext context,
       TableConfiguration tableConf, String tabletDirectory, Text prevEndRow, Text endRow,
-      Collection<String> mapFiles, double minSplit, boolean useIndex) throws IOException {
+      Collection<? extends TabletFile> dataFiles, double minSplit, boolean useIndex)
+      throws IOException {
 
-    Collection<String> origMapFiles = mapFiles;
+    Collection<? extends TabletFile> origDataFiles = dataFiles;
 
     Path tmpDir = null;
 
-    int maxToOpen =
-        context.getConfiguration().getCount(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN);
-    ArrayList<FileSKVIterator> readers = new ArrayList<>(mapFiles.size());
+    int maxToOpen = getMaxFilesToOpen(context.getConfiguration());
+    ArrayList<FileSKVIterator> readers = new ArrayList<>(dataFiles.size());
 
     try {
-      if (mapFiles.size() > maxToOpen) {
+      if (dataFiles.size() > maxToOpen) {
         if (!useIndex) {
           throw new IOException(
-              "Cannot find mid point using data files, too many " + mapFiles.size());
+              "Cannot find mid point using data files, too many " + dataFiles.size());
         }
         tmpDir = createTmpDir(context, tabletDirectory);
 
         log.debug("Too many indexes ({}) to open at once for {} {}, reducing in tmpDir = {}",
-            mapFiles.size(), endRow, prevEndRow, tmpDir);
+            dataFiles.size(), endRow, prevEndRow, tmpDir);
 
         long t1 = System.currentTimeMillis();
-        mapFiles =
-            reduceFiles(context, tableConf, prevEndRow, endRow, mapFiles, maxToOpen, tmpDir, 0);
+        dataFiles =
+            reduceFiles(context, tableConf, prevEndRow, endRow, dataFiles, maxToOpen, tmpDir, 0);
         long t2 = System.currentTimeMillis();
 
         log.debug("Finished reducing indexes for {} {} in {}", endRow, prevEndRow,
@@ -321,7 +329,7 @@
 
       long t1 = System.currentTimeMillis();
 
-      long numKeys = countIndexEntries(context, tableConf, prevEndRow, endRow, mapFiles,
+      long numKeys = countIndexEntries(context, tableConf, prevEndRow, endRow, dataFiles,
           tmpDir == null ? useIndex : false, readers);
 
       if (numKeys == 0) {
@@ -329,10 +337,10 @@
           log.warn(
               "Failed to find mid point using indexes, falling back to"
                   + " data files which is slower. No entries between {} and {} for {}",
-              prevEndRow, endRow, mapFiles);
-          // need to pass original map files, not possibly reduced indexes
-          return findMidPoint(context, tableConf, tabletDirectory, prevEndRow, endRow, origMapFiles,
-              minSplit, false);
+              prevEndRow, endRow, dataFiles);
+          // need to pass original data files, not possibly reduced indexes
+          return findMidPoint(context, tableConf, tabletDirectory, prevEndRow, endRow,
+              origDataFiles, minSplit, false);
         }
         return Collections.emptySortedMap();
       }
@@ -389,7 +397,7 @@
             (key.compareRow(prevEndRow) > 0 && (endRow == null || key.compareRow(endRow) < 1));
         if (!inRange) {
           throw new IOException("Found mid point is not in range " + key + " " + prevEndRow + " "
-              + endRow + " " + mapFiles);
+              + endRow + " " + dataFiles);
         }
       }
 
@@ -425,23 +433,22 @@
   }
 
   private static long countIndexEntries(ServerContext context, TableConfiguration tableConf,
-      Text prevEndRow, Text endRow, Collection<String> mapFiles, boolean useIndex,
+      Text prevEndRow, Text endRow, Collection<? extends TabletFile> dataFiles, boolean useIndex,
       ArrayList<FileSKVIterator> readers) throws IOException {
     long numKeys = 0;
 
     // count the total number of index entries
-    for (String file : mapFiles) {
+    for (TabletFile file : dataFiles) {
       FileSKVIterator reader = null;
-      Path path = new Path(file);
-      FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
+      FileSystem ns = context.getVolumeManager().getFileSystemByPath(file.getPath());
       try {
         if (useIndex) {
           reader = FileOperations.getInstance().newIndexReaderBuilder()
-              .forFile(path.toString(), ns, ns.getConf(), tableConf.getCryptoService())
+              .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
               .withTableConfiguration(tableConf).build();
         } else {
           reader = FileOperations.getInstance().newScanReaderBuilder()
-              .forFile(path.toString(), ns, ns.getConf(), tableConf.getCryptoService())
+              .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
               .withTableConfiguration(tableConf)
               .overRange(new Range(prevEndRow, false, null, true), Set.of(), false).build();
         }
@@ -468,11 +475,11 @@
 
       if (useIndex) {
         readers.add(FileOperations.getInstance().newIndexReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf(), tableConf.getCryptoService())
+            .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
             .withTableConfiguration(tableConf).build());
       } else {
         readers.add(FileOperations.getInstance().newScanReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf(), tableConf.getCryptoService())
+            .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
             .withTableConfiguration(tableConf)
             .overRange(new Range(prevEndRow, false, null, true), Set.of(), false).build());
       }
@@ -481,35 +488,35 @@
     return numKeys;
   }
 
-  public static Map<TabletFile,FileInfo> tryToGetFirstAndLastRows(ServerContext context,
-      TableConfiguration tableConf, Set<TabletFile> mapfiles) {
+  public static <T extends TabletFile> Map<T,FileInfo> tryToGetFirstAndLastRows(
+      ServerContext context, TableConfiguration tableConf, Set<T> dataFiles) {
 
-    HashMap<TabletFile,FileInfo> mapFilesInfo = new HashMap<>();
+    HashMap<T,FileInfo> dataFilesInfo = new HashMap<>();
 
     long t1 = System.currentTimeMillis();
 
-    for (TabletFile mapfile : mapfiles) {
+    for (T dataFile : dataFiles) {
 
       FileSKVIterator reader = null;
-      FileSystem ns = context.getVolumeManager().getFileSystemByPath(mapfile.getPath());
+      FileSystem ns = context.getVolumeManager().getFileSystemByPath(dataFile.getPath());
       try {
         reader = FileOperations.getInstance().newReaderBuilder()
-            .forFile(mapfile.getPathStr(), ns, ns.getConf(), tableConf.getCryptoService())
+            .forFile(dataFile, ns, ns.getConf(), tableConf.getCryptoService())
             .withTableConfiguration(tableConf).build();
 
-        Key firstKey = reader.getFirstKey();
-        if (firstKey != null) {
-          mapFilesInfo.put(mapfile, new FileInfo(firstKey, reader.getLastKey()));
+        Text firstRow = reader.getFirstRow();
+        if (firstRow != null) {
+          dataFilesInfo.put(dataFile, new FileInfo(firstRow, reader.getLastRow()));
         }
 
       } catch (IOException ioe) {
-        log.warn("Failed to read map file to determine first and last key : " + mapfile, ioe);
+        log.warn("Failed to read data file to determine first and last key : " + dataFile, ioe);
       } finally {
         if (reader != null) {
           try {
             reader.close();
           } catch (IOException ioe) {
-            log.warn("failed to close " + mapfile, ioe);
+            log.warn("failed to close " + dataFile, ioe);
           }
         }
       }
@@ -518,21 +525,21 @@
 
     long t2 = System.currentTimeMillis();
 
-    log.debug(String.format("Found first and last keys for %d map files in %6.2f secs",
-        mapfiles.size(), (t2 - t1) / 1000.0));
+    log.debug(String.format("Found first and last keys for %d data files in %6.2f secs",
+        dataFiles.size(), (t2 - t1) / 1000.0));
 
-    return mapFilesInfo;
+    return dataFilesInfo;
   }
 
-  public static WritableComparable<Key> findLastKey(ServerContext context,
-      TableConfiguration tableConf, Collection<TabletFile> mapFiles) throws IOException {
+  public static <T extends TabletFile> Text findLastRow(ServerContext context,
+      TableConfiguration tableConf, Collection<T> dataFiles) throws IOException {
 
-    Key lastKey = null;
+    Text lastRow = null;
 
-    for (TabletFile file : mapFiles) {
+    for (T file : dataFiles) {
       FileSystem ns = context.getVolumeManager().getFileSystemByPath(file.getPath());
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.getPathStr(), ns, ns.getConf(), tableConf.getCryptoService())
+          .forFile(file, ns, ns.getConf(), tableConf.getCryptoService())
           .withTableConfiguration(tableConf).seekToBeginning().build();
 
       try {
@@ -541,10 +548,10 @@
           continue;
         }
 
-        Key key = reader.getLastKey();
+        Text row = reader.getLastRow();
 
-        if (lastKey == null || key.compareTo(lastKey) > 0) {
-          lastKey = key;
+        if (lastRow == null || row.compareTo(lastRow) > 0) {
+          lastRow = row;
         }
       } finally {
         try {
@@ -557,7 +564,7 @@
       }
     }
 
-    return lastKey;
+    return lastRow;
 
   }
 
@@ -565,7 +572,8 @@
    * Convert TabletFiles to Strings in case we need to reduce number of files. The temporary files
    * used will have irregular paths that don't conform to TabletFile verification.
    */
-  public static Collection<String> toPathStrings(Collection<TabletFile> files) {
-    return files.stream().map(TabletFile::getPathStr).collect(Collectors.toList());
+  public static Collection<String> toPathStrings(Collection<ReferencedTabletFile> files) {
+    return files.stream().map(ReferencedTabletFile::getNormalizedPathStr)
+        .collect(Collectors.toList());
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
index 6e978a5..342fd82 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FindOfflineTablets.java
@@ -27,8 +27,7 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 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 org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletState;
@@ -91,22 +90,22 @@
       return offline;
     }
 
-    if (RootTable.NAME.equals(tableName)) {
+    if (AccumuloTable.ROOT.tableName().equals(tableName)) {
       return 0;
     }
 
-    System.out.println("Scanning " + RootTable.NAME);
-    Iterator<TabletLocationState> rootScanner =
-        new MetaDataTableScanner(context, TabletsSection.getRange(), RootTable.NAME);
+    System.out.println("Scanning " + AccumuloTable.ROOT.tableName());
+    Iterator<TabletLocationState> rootScanner = new MetaDataTableScanner(context,
+        TabletsSection.getRange(), AccumuloTable.ROOT.tableName());
     if ((offline = checkTablets(context, rootScanner, tservers)) > 0) {
       return offline;
     }
 
-    if (MetadataTable.NAME.equals(tableName)) {
+    if (AccumuloTable.METADATA.tableName().equals(tableName)) {
       return 0;
     }
 
-    System.out.println("Scanning " + MetadataTable.NAME);
+    System.out.println("Scanning " + AccumuloTable.METADATA.tableName());
 
     Range range = TabletsSection.getRange();
     if (tableName != null) {
@@ -115,7 +114,7 @@
     }
 
     try (MetaDataTableScanner metaScanner =
-        new MetaDataTableScanner(context, range, MetadataTable.NAME)) {
+        new MetaDataTableScanner(context, range, AccumuloTable.METADATA.tableName())) {
       return checkTablets(context, metaScanner, tservers);
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
index 131ded3..275e11e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
@@ -24,6 +24,7 @@
 import java.util.Formatter;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
@@ -32,9 +33,11 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -164,11 +167,11 @@
     try {
       var zLockManagerPath =
           ServiceLock.path(Constants.ZROOT + "/" + iid + Constants.ZMANAGER_LOCK);
-      byte[] manager = ServiceLock.getLockData(cache, zLockManagerPath, null);
-      if (manager == null) {
+      Optional<ServiceLockData> sld = ServiceLock.getLockData(cache, zLockManagerPath, null);
+      if (sld.isEmpty()) {
         return null;
       }
-      return new String(manager, UTF_8);
+      return sld.orElseThrow().getAddressString(ThriftService.MANAGER);
     } catch (Exception e) {
       handleException(e, printErrors);
       return null;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index 060b87d..30b3d1d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -55,7 +55,7 @@
   }
 
   private static void getLogURIs(TreeSet<String> volumes, LogEntry logEntry) {
-    volumes.add(getLogURI(logEntry.filename));
+    volumes.add(getLogURI(logEntry.getPath()));
   }
 
   private static void listTable(Ample.DataLevel level, ServerContext context) throws Exception {
@@ -66,7 +66,8 @@
     try (TabletsMetadata tablets = TabletsMetadata.builder(context).forLevel(level)
         .fetch(TabletMetadata.ColumnType.FILES, TabletMetadata.ColumnType.LOGS).build()) {
       for (TabletMetadata tabletMetadata : tablets) {
-        tabletMetadata.getFiles().forEach(file -> volumes.add(getTableURI(file.getPathStr())));
+        tabletMetadata.getFiles()
+            .forEach(file -> volumes.add(getTableURI(file.getNormalizedPathStr())));
         tabletMetadata.getLogs().forEach(le -> getLogURIs(volumes, le));
       }
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
index c0b5e27..eb179bd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java
@@ -40,10 +40,10 @@
 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.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 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.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
@@ -54,6 +54,7 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -64,10 +65,13 @@
 public class ManagerMetadataUtil {
 
   private static final Logger log = LoggerFactory.getLogger(ManagerMetadataUtil.class);
+  @SuppressWarnings("deprecation")
+  private static final Property DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY =
+      Property.TSERV_LAST_LOCATION_MODE;
 
   public static void addNewTablet(ServerContext context, KeyExtent extent, String dirName,
       TServerInstance tServerInstance, Map<StoredTabletFile,DataFileValue> datafileSizes,
-      Map<Long,? extends Collection<TabletFile>> bulkLoadedFiles, MetadataTime time,
+      Map<Long,? extends Collection<ReferencedTabletFile>> bulkLoadedFiles, MetadataTime time,
       long lastFlushID, long lastCompactID, ServiceLock zooLock) {
 
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
@@ -89,10 +93,11 @@
       tablet.deleteLocation(Location.future(tServerInstance));
     }
 
-    datafileSizes.forEach(tablet::putFile);
+    datafileSizes.forEach((key, value) -> tablet.putFile(key.getTabletFile(), value));
 
-    for (Entry<Long,? extends Collection<TabletFile>> entry : bulkLoadedFiles.entrySet()) {
-      for (TabletFile ref : entry.getValue()) {
+    for (Entry<Long,? extends Collection<ReferencedTabletFile>> entry : bulkLoadedFiles
+        .entrySet()) {
+      for (ReferencedTabletFile ref : entry.getValue()) {
         tablet.putBulkFile(ref, entry.getKey());
       }
     }
@@ -215,8 +220,8 @@
    * Update tablet file data from flush. Returns a StoredTabletFile if there are data entries.
    */
   public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext context,
-      KeyExtent extent, TabletFile newDatafile, DataFileValue dfv, MetadataTime time,
-      TServerInstance tServerInstance, ServiceLock zooLock, Set<String> unusedWalLogs,
+      KeyExtent extent, ReferencedTabletFile newDatafile, DataFileValue dfv, MetadataTime time,
+      TServerInstance tServerInstance, ServiceLock zooLock, Set<LogEntry> unusedWalLogs,
       Location lastLocation, long flushId) {
 
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
@@ -257,7 +262,8 @@
 
     // if the location mode is assignment, then preserve the current location in the last
     // location value
-    if ("assignment".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE))) {
+    if ("assignment"
+        .equals(context.getConfiguration().get(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY))) {
       ManagerMetadataUtil.updateLocation(tabletMutator, lastLocation, Location.last(location));
     }
   }
@@ -275,7 +281,8 @@
       Location lastLocation, TServerInstance tServerInstance) {
     // if the location mode is 'compaction', then preserve the current compaction location in the
     // last location value
-    if ("compaction".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE))) {
+    if ("compaction"
+        .equals(context.getConfiguration().get(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY))) {
       Location newLocation = Location.last(tServerInstance);
       updateLocation(tabletMutator, lastLocation, newLocation);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 60b16e5..9209f82 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.server.util;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.CLONED;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.DIR;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
@@ -26,7 +27,6 @@
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.TIME;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -58,18 +58,16 @@
 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.gc.ReferenceFile;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+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.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-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;
@@ -156,7 +154,7 @@
   }
 
   public static Map<StoredTabletFile,DataFileValue> updateTabletDataFile(long tid, KeyExtent extent,
-      Map<TabletFile,DataFileValue> estSizes, MetadataTime time, ServerContext context,
+      Map<ReferencedTabletFile,DataFileValue> estSizes, MetadataTime time, ServerContext context,
       ServiceLock zooLock) {
     TabletMutator tablet = context.getAmple().mutateTablet(extent);
     tablet.putTime(time);
@@ -185,7 +183,8 @@
 
   public static void updateTabletVolumes(KeyExtent extent, List<LogEntry> logsToRemove,
       List<LogEntry> logsToAdd, List<StoredTabletFile> filesToRemove,
-      SortedMap<TabletFile,DataFileValue> filesToAdd, ServiceLock zooLock, ServerContext context) {
+      SortedMap<ReferencedTabletFile,DataFileValue> filesToAdd, ServiceLock zooLock,
+      ServerContext context) {
 
     TabletMutator tabletMutator = context.getAmple().mutateTablet(extent);
     logsToRemove.forEach(tabletMutator::deleteWal);
@@ -216,7 +215,6 @@
 
     TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m,
         TabletColumnFamily.encodePrevEndRow(oldPrevEndRow));
-    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 
     ecids.forEach(ecid -> m.putDelete(ExternalCompactionColumnFamily.STR_NAME, ecid.canonical()));
 
@@ -230,15 +228,14 @@
     Mutation m = new Mutation(metadataEntry);
     TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
     TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
-    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 
     for (Entry<StoredTabletFile,DataFileValue> entry : datafileSizes.entrySet()) {
-      m.put(DataFileColumnFamily.NAME, entry.getKey().getMetaUpdateDeleteText(),
+      m.put(DataFileColumnFamily.NAME, entry.getKey().getMetadataText(),
           new Value(entry.getValue().encode()));
     }
 
     for (StoredTabletFile pathToRemove : highDatafilesToRemove) {
-      m.putDelete(DataFileColumnFamily.NAME, pathToRemove.getMetaUpdateDeleteText());
+      m.putDelete(DataFileColumnFamily.NAME, pathToRemove.getMetadataText());
     }
 
     update(context, zooLock, m, KeyExtent.fromMetaRow(metadataEntry));
@@ -259,7 +256,7 @@
   }
 
   public static void splitDatafiles(Text midRow, double splitRatio,
-      Map<TabletFile,FileUtil.FileInfo> firstAndLastRows,
+      Map<StoredTabletFile,FileUtil.FileInfo> firstAndLastRows,
       SortedMap<StoredTabletFile,DataFileValue> datafiles,
       SortedMap<StoredTabletFile,DataFileValue> lowDatafileSizes,
       SortedMap<StoredTabletFile,DataFileValue> highDatafileSizes,
@@ -311,8 +308,10 @@
 
   public static void deleteTable(TableId tableId, boolean insertDeletes, ServerContext context,
       ServiceLock lock) throws AccumuloException {
-    try (Scanner ms = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY);
-        BatchWriter bw = new BatchWriterImpl(context, MetadataTable.ID,
+    try (
+        Scanner ms =
+            new ScannerImpl(context, AccumuloTable.METADATA.tableId(), Authorizations.EMPTY);
+        BatchWriter bw = new BatchWriterImpl(context, AccumuloTable.METADATA.tableId(),
             new BatchWriterConfig().setMaxMemory(1000000)
                 .setMaxLatency(120000L, TimeUnit.MILLISECONDS).setMaxWriteThreads(2))) {
 
@@ -332,8 +331,7 @@
 
           if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
             StoredTabletFile stf = new StoredTabletFile(key.getColumnQualifierData().toString());
-            bw.addMutation(ample
-                .createDeleteMutation(ReferenceFile.forFile(tableId, stf.getMetaUpdateDelete())));
+            bw.addMutation(ample.createDeleteMutation(ReferenceFile.forFile(tableId, stf)));
           }
 
           if (ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
@@ -381,7 +379,7 @@
     TabletMetadata tablet = context.getAmple().readTablet(extent, FILES, LOGS, PREV_ROW, DIR);
 
     if (tablet == null) {
-      throw new RuntimeException("Tablet " + extent + " not found in metadata");
+      throw new IllegalStateException("Tablet " + extent + " not found in metadata");
     }
 
     result.addAll(tablet.getLogs());
@@ -433,11 +431,11 @@
     if (testTableName != null) {
       tableName = testTableName;
       range = TabletsSection.getRange(tableId);
-    } else if (tableId.equals(MetadataTable.ID)) {
-      tableName = RootTable.NAME;
+    } else if (tableId.equals(AccumuloTable.METADATA.tableId())) {
+      tableName = AccumuloTable.ROOT.tableName();
       range = TabletsSection.getRange();
     } else {
-      tableName = MetadataTable.NAME;
+      tableName = AccumuloTable.METADATA.tableName();
       range = TabletsSection.getRange(tableId);
     }
 
@@ -453,7 +451,7 @@
     Iterator<TabletMetadata> ti = createCloneScanner(testTableName, srcTableId, client).iterator();
 
     if (!ti.hasNext()) {
-      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
+      throw new IllegalStateException(" table deleted during clone?  srcTableId = " + srcTableId);
     }
 
     while (ti.hasNext()) {
@@ -479,7 +477,7 @@
         createCloneScanner(testTableName, tableId, client).iterator();
 
     if (!cloneIter.hasNext() || !srcIter.hasNext()) {
-      throw new RuntimeException(
+      throw new IllegalStateException(
           " table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
     }
 
@@ -488,7 +486,7 @@
     while (cloneIter.hasNext()) {
       TabletMetadata cloneTablet = cloneIter.next();
       Text cloneEndRow = cloneTablet.getEndRow();
-      HashSet<TabletFile> cloneFiles = new HashSet<>();
+      HashSet<StoredTabletFile> cloneFiles = new HashSet<>();
 
       boolean cloneSuccessful = cloneTablet.getCloned() != null;
 
@@ -507,7 +505,7 @@
             "Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
       }
 
-      HashSet<TabletFile> srcFiles = new HashSet<>();
+      HashSet<StoredTabletFile> srcFiles = new HashSet<>();
       if (!cloneSuccessful) {
         srcFiles.addAll(srcTablet.getFiles());
       }
@@ -562,7 +560,7 @@
   public static void cloneTable(ServerContext context, TableId srcTableId, TableId tableId)
       throws Exception {
 
-    try (BatchWriter bw = context.createBatchWriter(MetadataTable.NAME)) {
+    try (BatchWriter bw = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
 
       while (true) {
 
@@ -598,7 +596,8 @@
       }
 
       // delete the clone markers and create directory entries
-      Scanner mscanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      Scanner mscanner =
+          context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
       mscanner.setRange(new KeyExtent(tableId, null, null).toMetaRange());
       mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 
@@ -617,11 +616,4 @@
     }
   }
 
-  public static void chopped(ServerContext context, KeyExtent extent, ServiceLock zooLock) {
-    TabletMutator tablet = context.getAmple().mutateTablet(extent);
-    tablet.putChopped();
-    tablet.putZooLock(zooLock);
-    tablet.mutate();
-  }
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/PortUtils.java b/server/base/src/main/java/org/apache/accumulo/server/util/PortUtils.java
index e6f653a..cdfd6b8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/PortUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/PortUtils.java
@@ -49,6 +49,6 @@
       count++;
     }
 
-    throw new RuntimeException("Unable to find port");
+    throw new IllegalStateException("Unable to find port");
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java b/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
index 2d347f8..4bc52c3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RandomWriter.java
@@ -18,7 +18,8 @@
  */
 package org.apache.accumulo.server.util;
 
-import java.security.SecureRandom;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.util.Iterator;
 import java.util.Properties;
 
@@ -44,7 +45,6 @@
   private static int num_columns_per_row = 1;
   private static int num_payload_bytes = 1024;
   private static final Logger log = LoggerFactory.getLogger(RandomWriter.class);
-  private static final SecureRandom random = new SecureRandom();
 
   public static class RandomMutationGenerator implements Iterable<Mutation>, Iterator<Mutation> {
     private long max_mutations;
@@ -63,12 +63,12 @@
     @Override
     public Mutation next() {
       Text row_value = new Text(
-          Long.toString(((random.nextLong() & 0x7fffffffffffffffL) / 177) % 100000000000L));
+          Long.toString(((RANDOM.get().nextLong() & 0x7fffffffffffffffL) / 177) % 100000000000L));
       Mutation m = new Mutation(row_value);
       for (int column = 0; column < num_columns_per_row; column++) {
         Text column_fam = new Text("col_fam");
         byte[] bytes = new byte[num_payload_bytes];
-        random.nextBytes(bytes);
+        RANDOM.get().nextBytes(bytes);
         m.put(column_fam, new Text("" + column), new Value(bytes));
       }
       mutations_so_far++;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
index 18de106..19c11ba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
@@ -35,9 +35,8 @@
 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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.ValidationUtil;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
@@ -55,7 +54,7 @@
 import io.opentelemetry.context.Scope;
 
 /**
- * Remove file entries for map files that don't exist.
+ * Remove file entries for data files that don't exist.
  */
 public class RemoveEntriesForMissingFiles {
 
@@ -136,7 +135,7 @@
     BatchWriter writer = null;
 
     if (fix) {
-      writer = context.createBatchWriter(MetadataTable.NAME);
+      writer = context.createBatchWriter(AccumuloTable.METADATA.tableName());
     }
 
     for (Entry<Key,Value> entry : metadata) {
@@ -146,7 +145,7 @@
 
       count++;
       Key key = entry.getKey();
-      Path map = new Path(ValidationUtil.validate(key.getColumnQualifierData().toString()));
+      Path map = StoredTabletFile.of(key.getColumnQualifierData().toString()).getPath();
 
       synchronized (processing) {
         while (processing.size() >= 64 || processing.contains(map)) {
@@ -186,24 +185,26 @@
   }
 
   static int checkAllTables(ServerContext context, boolean fix) throws Exception {
-    int missing = checkTable(context, RootTable.NAME, TabletsSection.getRange(), fix);
+    int missing =
+        checkTable(context, AccumuloTable.ROOT.tableName(), TabletsSection.getRange(), fix);
 
     if (missing == 0) {
-      return checkTable(context, MetadataTable.NAME, TabletsSection.getRange(), fix);
+      return checkTable(context, AccumuloTable.METADATA.tableName(), TabletsSection.getRange(),
+          fix);
     } else {
       return missing;
     }
   }
 
   static int checkTable(ServerContext context, String tableName, boolean fix) throws Exception {
-    if (tableName.equals(RootTable.NAME)) {
+    if (tableName.equals(AccumuloTable.ROOT.tableName())) {
       throw new IllegalArgumentException("Can not check root table");
-    } else if (tableName.equals(MetadataTable.NAME)) {
-      return checkTable(context, RootTable.NAME, TabletsSection.getRange(), fix);
+    } else if (tableName.equals(AccumuloTable.METADATA.tableName())) {
+      return checkTable(context, AccumuloTable.ROOT.tableName(), TabletsSection.getRange(), fix);
     } else {
       TableId tableId = context.getTableId(tableName);
       Range range = new KeyExtent(tableId, null, null).toMetaRange();
-      return checkTable(context, MetadataTable.NAME, range, fix);
+      return checkTable(context, AccumuloTable.METADATA.tableName(), range, fix);
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
deleted file mode 100644
index 70f7845..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
+++ /dev/null
@@ -1,193 +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.server.util;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
-
-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.IteratorSetting;
-import org.apache.accumulo.core.client.IteratorSetting.Column;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.Writer;
-import org.apache.accumulo.core.conf.Property;
-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.iterators.Combiner;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * provides a reference to the replication table for updates by tablet servers
- */
-public class ReplicationTableUtil {
-
-  private static Map<Credentials,Writer> writers = new HashMap<>();
-  private static final Logger log = LoggerFactory.getLogger(ReplicationTableUtil.class);
-
-  public static final String COMBINER_NAME = "replcombiner";
-  @SuppressWarnings("deprecation")
-  public static final String STATUS_FORMATTER_CLASS_NAME =
-      org.apache.accumulo.server.replication.StatusFormatter.class.getName();
-
-  private ReplicationTableUtil() {}
-
-  /**
-   * For testing purposes only -- should not be called by server code
-   * <p>
-   * Allows mocking of a Writer for testing
-   *
-   * @param creds Credentials
-   * @param writer A Writer to use for the given credentials
-   */
-  static synchronized void addWriter(Credentials creds, Writer writer) {
-    writers.put(creds, writer);
-  }
-
-  static synchronized Writer getWriter(ClientContext context) {
-    Writer replicationTable = writers.get(context.getCredentials());
-    if (replicationTable == null) {
-      configureMetadataTable(context, MetadataTable.NAME);
-      replicationTable = new Writer(context, MetadataTable.ID);
-      writers.put(context.getCredentials(), replicationTable);
-    }
-    return replicationTable;
-  }
-
-  public synchronized static void configureMetadataTable(AccumuloClient client, String tableName) {
-    TableOperations tops = client.tableOperations();
-    Map<String,EnumSet<IteratorScope>> iterators = null;
-    try {
-      iterators = tops.listIterators(tableName);
-    } catch (AccumuloSecurityException | AccumuloException | TableNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-
-    if (!iterators.containsKey(COMBINER_NAME)) {
-      // Set our combiner and combine all columns
-      // Need to set the combiner beneath versioning since we don't want to turn it off
-      @SuppressWarnings("deprecation")
-      var statusCombinerClass = org.apache.accumulo.server.replication.StatusCombiner.class;
-      IteratorSetting setting = new IteratorSetting(9, COMBINER_NAME, statusCombinerClass);
-      Combiner.setColumns(setting, Collections.singletonList(new Column(ReplicationSection.COLF)));
-      try {
-        tops.attachIterator(tableName, setting);
-      } catch (AccumuloSecurityException | AccumuloException | TableNotFoundException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    // Make sure the StatusFormatter is set on the metadata table
-    Map<String,String> properties;
-    try {
-      properties = tops.getConfiguration(tableName);
-    } catch (AccumuloException | TableNotFoundException e) {
-      throw new RuntimeException(e);
-    }
-
-    for (Entry<String,String> property : properties.entrySet()) {
-      if (Property.TABLE_FORMATTER_CLASS.getKey().equals(property.getKey())) {
-        if (!STATUS_FORMATTER_CLASS_NAME.equals(property.getValue())) {
-          log.info("Setting formatter for {} from {} to {}", tableName, property.getValue(),
-              STATUS_FORMATTER_CLASS_NAME);
-          try {
-            tops.setProperty(tableName, Property.TABLE_FORMATTER_CLASS.getKey(),
-                STATUS_FORMATTER_CLASS_NAME);
-          } catch (AccumuloException | AccumuloSecurityException e) {
-            throw new RuntimeException(e);
-          }
-        }
-
-        // Don't need to keep iterating over the properties after we found the one we were looking
-        // for
-        return;
-      }
-    }
-
-    // Set the formatter on the table because it wasn't already there
-    try {
-      tops.setProperty(tableName, Property.TABLE_FORMATTER_CLASS.getKey(),
-          STATUS_FORMATTER_CLASS_NAME);
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Write the given Mutation to the replication table.
-   */
-  static void update(ClientContext context, Mutation m) {
-    Writer t = getWriter(context);
-    while (true) {
-      try {
-        t.update(m);
-        return;
-      } catch (AccumuloException | TableNotFoundException e) {
-        log.error(e.toString(), e);
-      }
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
-    }
-  }
-
-  /**
-   * Write replication ingest entries for each provided file with the given {@link Status}.
-   */
-  public static void updateFiles(ClientContext context, KeyExtent extent, String file,
-      Status stat) {
-    if (log.isDebugEnabled()) {
-      log.debug("Updating replication status for {} with {} using {}", extent, file,
-          ProtobufUtil.toString(stat));
-    }
-    // TODO could use batch writer, would need to handle failure and retry like update does -
-    // ACCUMULO-1294
-
-    Value v = ProtobufUtil.toValue(stat);
-    update(context, createUpdateMutation(new Path(file), v, extent));
-  }
-
-  static Mutation createUpdateMutation(Path file, Value v, KeyExtent extent) {
-    // Need to normalize the file path so we can assuredly find it again later
-    return createUpdateMutation(new Text(ReplicationSection.getRowPrefix() + file), v, extent);
-  }
-
-  private static Mutation createUpdateMutation(Text row, Value v, KeyExtent extent) {
-    Mutation m = new Mutation(row);
-    m.put(ReplicationSection.COLF, new Text(extent.tableId().canonical()), v);
-    return m;
-  }
-}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
index 6d6e13c..ff09cd3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
@@ -55,7 +55,7 @@
       if ("node".equals(name)) {
         String child = attributes.getValue("name");
         if (child == null) {
-          throw new RuntimeException("name attribute not set");
+          throw new IllegalStateException("name attribute not set");
         }
         String encoding = attributes.getValue("encoding");
         String value = attributes.getValue("value");
@@ -95,12 +95,12 @@
               overwrite ? NodeExistsPolicy.OVERWRITE : NodeExistsPolicy.FAIL);
         } catch (KeeperException e) {
           if (e.code().equals(KeeperException.Code.NODEEXISTS)) {
-            throw new RuntimeException(path + " exists.  Remove it first.");
+            throw new IllegalStateException(path + " exists.  Remove it first.");
           }
           throw e;
         }
       } catch (Exception e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java b/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java
index 3f54cf9..1ab394b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java
@@ -23,8 +23,8 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
 
 // A little class to hold bulk import status information in the Manager
 // and two places in the tablet server.
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
index 8eed2cc..ee670a2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
@@ -43,16 +43,14 @@
 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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.NumUtil;
 import org.apache.accumulo.server.cli.ServerUtilOpts;
-import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -205,18 +203,18 @@
     for (TableId tableId : tableIds) {
       // if the table to compute usage is for the metadata table itself then we need to scan the
       // root table, else we scan the metadata table
-      try (Scanner mdScanner = tableId.equals(MetadataTable.ID)
-          ? client.createScanner(RootTable.NAME, Authorizations.EMPTY)
-          : client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner mdScanner = tableId.equals(AccumuloTable.METADATA.tableId())
+          ? client.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY)
+          : client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         mdScanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
         mdScanner.setRange(new KeyExtent(tableId, null, null).toMetaRange());
 
-        final Set<TabletFile> files = new HashSet<>();
+        final Set<StoredTabletFile> files = new HashSet<>();
 
         // Read each file referenced by that table
         for (Map.Entry<Key,Value> entry : mdScanner) {
-          final TabletFile file =
-              new TabletFile(new Path(entry.getKey().getColumnQualifier().toString()));
+          final StoredTabletFile file =
+              new StoredTabletFile(entry.getKey().getColumnQualifier().toString());
 
           // get the table referenced by the file which may not be the same as the current
           // table we are scanning if the file is shared between multiple tables
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TableInfoUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/TableInfoUtil.java
index 35f772e..888d5e2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TableInfoUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TableInfoUtil.java
@@ -21,10 +21,10 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.accumulo.core.manager.thrift.Compacting;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.Compacting;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 
 public class TableInfoUtil {
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
index e55237d..8afb863 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
@@ -18,14 +18,15 @@
  */
 package org.apache.accumulo.server.util;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-
 import java.util.List;
+import java.util.Optional;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+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.server.ServerContext;
 
 public class TabletServerLocks {
@@ -45,10 +46,10 @@
 
       for (String tabletServer : tabletServers) {
         var zLockPath = ServiceLock.path(tserverPath + "/" + tabletServer);
-        byte[] lockData = ServiceLock.getLockData(cache, zLockPath, null);
+        Optional<ServiceLockData> lockData = ServiceLock.getLockData(cache, zLockPath, null);
         final String holder;
-        if (lockData != null) {
-          holder = new String(lockData, UTF_8);
+        if (lockData.isPresent()) {
+          holder = lockData.orElseThrow().getAddressString(ThriftService.TSERV);
         } else {
           holder = "<none>";
         }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
index f120183..a83f746 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/VerifyTabletAssignments.java
@@ -35,6 +35,7 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
@@ -48,10 +49,9 @@
 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.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 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.HostAndPortComparator;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
@@ -59,6 +59,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;
 
@@ -95,7 +97,7 @@
 
     final HashSet<KeyExtent> failures = new HashSet<>();
 
-    Map<HostAndPort,List<KeyExtent>> extentsPerServer = new TreeMap<>();
+    Map<HostAndPort,List<KeyExtent>> extentsPerServer = new TreeMap<>(new HostAndPortComparator());
 
     for (Entry<KeyExtent,String> entry : tabletLocations.entrySet()) {
       KeyExtent keyExtent = entry.getKey();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
index 8e37cba..3a2b617 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
@@ -25,9 +25,9 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 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.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
@@ -65,10 +65,6 @@
   }
 
   static class Opts extends Help {
-    @Deprecated(since = "2.1.0")
-    @Parameter(names = "-master",
-        description = "remove master locks (deprecated -- user -manager instead")
-    boolean zapMaster = false;
     @Parameter(names = "-manager", description = "remove manager locks")
     boolean zapManager = false;
     @Parameter(names = "-tservers", description = "remove tablet server locks")
@@ -93,7 +89,7 @@
     Opts opts = new Opts();
     opts.parseArgs(keyword(), args);
 
-    if (!opts.zapMaster && !opts.zapManager && !opts.zapTservers) {
+    if (!opts.zapManager && !opts.zapTservers) {
       new JCommander(opts).usage();
       return;
     }
@@ -110,10 +106,7 @@
       InstanceId iid = VolumeManager.getInstanceIDFromHdfs(instanceDir, new Configuration());
       ZooReaderWriter zoo = new ZooReaderWriter(siteConf);
 
-      if (opts.zapMaster) {
-        log.warn("The -master option is deprecated. Please use -manager instead.");
-      }
-      if (opts.zapManager || opts.zapMaster) {
+      if (opts.zapManager) {
         String managerLockPath = Constants.ZROOT + "/" + iid + Constants.ZMANAGER_LOCK;
 
         try {
@@ -130,7 +123,7 @@
           for (String child : children) {
             message("Deleting " + tserversPath + "/" + child + " from zookeeper", opts);
 
-            if (opts.zapManager || opts.zapMaster) {
+            if (opts.zapManager) {
               zoo.recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP);
             } else {
               var zLockPath = ServiceLock.path(tserversPath + "/" + child);
@@ -146,15 +139,6 @@
         }
       }
 
-      // Remove the tracers, we don't use them anymore.
-      @SuppressWarnings("deprecation")
-      String path = siteConf.get(Property.TRACE_ZK_PATH);
-      try {
-        zapDirectory(zoo, path, opts);
-      } catch (Exception e) {
-        // do nothing if the /tracers node does not exist.
-      }
-
       if (opts.zapCoordinators) {
         final String coordinatorPath = Constants.ZROOT + "/" + iid + Constants.ZCOORDINATOR_LOCK;
         try {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
index 0423906..cd66790 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
@@ -21,8 +21,8 @@
 import static java.lang.Math.toIntExact;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -53,7 +53,6 @@
  */
 public class DistributedWorkQueue {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final String LOCKS_NODE = "locks";
 
   private static final Logger log = LoggerFactory.getLogger(DistributedWorkQueue.class);
@@ -75,7 +74,7 @@
       return;
     }
 
-    Collections.shuffle(children, random);
+    Collections.shuffle(children, RANDOM.get());
     try {
       for (final String child : children) {
 
@@ -170,7 +169,7 @@
 
   public DistributedWorkQueue(String path, AccumuloConfiguration config, ServerContext context) {
     // Preserve the old delay and period
-    this(path, config, context, random.nextInt(toIntExact(MINUTES.toMillis(1))),
+    this(path, config, context, RANDOM.get().nextInt(toIntExact(MINUTES.toMillis(1))),
         MINUTES.toMillis(1));
   }
 
diff --git a/server/base/src/main/protobuf/replication.proto b/server/base/src/main/protobuf/replication.proto
deleted file mode 100644
index 07d7cb3..0000000
--- a/server/base/src/main/protobuf/replication.proto
+++ /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.
- */
-syntax = "proto2";
-option java_package = "org.apache.accumulo.server.replication.proto";
-option optimize_for = SPEED;
-
-message Status {
-	optional int64 begin = 1 [default = 0]; // offset where replication should start
-	optional int64 end = 2 [default = 0]; // offset where data is ready for replication
-	optional bool infiniteEnd = 3 [default = false]; // do we have a discrete 'end'
-	optional bool closed = 4 [default = false]; // will more data be appended to the file
-	optional int64 createdTime = 5 [default = 0]; // when, in ms, was the file created?
-}
diff --git a/server/base/src/main/scripts/generate-protobuf.sh b/server/base/src/main/scripts/generate-protobuf.sh
deleted file mode 100755
index 36884e0..0000000
--- a/server/base/src/main/scripts/generate-protobuf.sh
+++ /dev/null
@@ -1,103 +0,0 @@
-#! /usr/bin/env bash
-#
-# 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.
-#
-
-# This script will regenerate the protobuf code for Accumulo
-
-# NOTES:
-#   To support this script being called by other modules, only edit the right side.
-#   In other scripts, set the variables that diverge from the defaults below, then call this script.
-#   Leave the BUILD_DIR and FINAL_DIR alone for Maven builds.
-# ========================================================================================================================
-[[ -z $REQUIRED_PROTOC_VERSION ]] && REQUIRED_PROTOC_VERSION='libprotoc 3.19.2'
-[[ -z $BUILD_DIR ]] && BUILD_DIR='target/proto'
-[[ -z $FINAL_DIR ]] && FINAL_DIR='src/main'
-# ========================================================================================================================
-
-fail() {
-  echo "$@"
-  exit 1
-}
-
-# Test to see if we have protoc installed
-if ! protoc --version 2>/dev/null | grep -qF "${REQUIRED_PROTOC_VERSION}"; then
-  # Nope: bail
-  echo "****************************************************"
-  echo "*** protoc is not available"
-  echo "***   expecting 'protoc --version' to return ${REQUIRED_PROTOC_VERSION}"
-  echo "*** generated code will not be updated"
-  fail "****************************************************"
-fi
-
-# Ensure output directories are created
-rm -rf "$BUILD_DIR"
-mkdir -p "$BUILD_DIR"
-
-protoc --java_out="$BUILD_DIR" src/main/protobuf/*.proto || fail unable to generate Java protocol buffer classes
-
-# For all generated protobuf code, suppress all warnings and add the LICENSE header
-s='@SuppressWarnings("unused")'
-find "$BUILD_DIR" -name '*.java' -print0 | xargs -0 sed -i.orig -e 's/\(public final class \)/'"$s"' \1/'
-
-PREFIX="/*
-"
-LINE_NOTATION=" *"
-SUFFIX="
- */"
-FILE_SUFFIX=(.java)
-
-for file in "${FILE_SUFFIX[@]}"; do
-  mapfile -t ALL_FILES_TO_LICENSE < <(find "$BUILD_DIR/" -name "*$file")
-  for f in "${ALL_FILES_TO_LICENSE[@]}"; do
-    cat - "$f" >"${f}-with-license" <<EOF
-${PREFIX}${LINE_NOTATION} Licensed to the Apache Software Foundation (ASF) under one
-${LINE_NOTATION} or more contributor license agreements.  See the NOTICE file
-${LINE_NOTATION} distributed with this work for additional information
-${LINE_NOTATION} regarding copyright ownership.  The ASF licenses this file
-${LINE_NOTATION} to you under the Apache License, Version 2.0 (the
-${LINE_NOTATION} "License"); you may not use this file except in compliance
-${LINE_NOTATION} with the License.  You may obtain a copy of the License at
-${LINE_NOTATION}
-${LINE_NOTATION}   https://www.apache.org/licenses/LICENSE-2.0
-${LINE_NOTATION}
-${LINE_NOTATION} Unless required by applicable law or agreed to in writing,
-${LINE_NOTATION} software distributed under the License is distributed on an
-${LINE_NOTATION} "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-${LINE_NOTATION} KIND, either express or implied.  See the License for the
-${LINE_NOTATION} specific language governing permissions and limitations
-${LINE_NOTATION} under the License.${SUFFIX}
-EOF
-  done
-done
-
-# For every generated java file, compare it with the version-controlled one, and copy the ones that have changed into place
-SDIR="${BUILD_DIR}/org/apache/accumulo/server/replication/proto"
-DDIR="${FINAL_DIR}/java/org/apache/accumulo/server/replication/proto"
-FILE_SUFFIX=(.java)
-mkdir -p "$DDIR"
-for file in "${FILE_SUFFIX[@]}"; do
-  mapfile -t ALL_LICENSE_FILES_TO_COPY < <(find "$SDIR" -name "*$file")
-  for f in "${ALL_LICENSE_FILES_TO_COPY[@]}"; do
-    DEST=$DDIR/$(basename "$f")
-    if ! cmp -s "${f}-with-license" "${DEST}"; then
-      echo cp -f "${f}-with-license" "${DEST}"
-      cp -f "${f}-with-license" "${DEST}" || fail unable to copy files to java workspace
-    fi
-  done
-done
diff --git a/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java b/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java
index c35a143..801e0c3 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/MockServerContext.java
@@ -49,6 +49,7 @@
     var sc = get();
     expect(sc.getZooKeeperRoot()).andReturn("/accumulo/" + instanceID).anyTimes();
     expect(sc.getInstanceID()).andReturn(instanceID).anyTimes();
+    expect(sc.zkUserPath()).andReturn("/accumulo/" + instanceID + "/users").anyTimes();
     expect(sc.getZooKeepers()).andReturn(zk).anyTimes();
     expect(sc.getZooKeepersSessionTimeOut()).andReturn(zkTimeout).anyTimes();
     return sc;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java
index 674d48e..fe04b78 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/ServerContextTest.java
@@ -115,12 +115,13 @@
       }).anyTimes();
 
       expect(siteConfig.iterator()).andAnswer(conf::iterator).anyTimes();
+      expect(siteConfig.stream()).andAnswer(conf::stream).anyTimes();
 
       replay(factory, context, siteConfig);
 
       assertEquals(ThriftServerType.SASL, context.getThriftServerType());
       SaslServerConnectionParams saslParams = context.getSaslParams();
-      assertEquals(new SaslServerConnectionParams(conf, token), saslParams);
+      assertEquals(new SaslServerConnectionParams(conf, token, null), saslParams);
       assertEquals(username, saslParams.getPrincipal());
 
       verify(factory, context, siteConfig);
@@ -129,14 +130,13 @@
     });
   }
 
+  // ensure upgrades fail with older, unsupported versions, but pass with supported versions
   @Test
   public void testCanRun() {
-    // ensure this fails with older versions; the oldest supported version is hard-coded here
-    // to ensure we don't unintentionally break upgrade support; changing this should be a conscious
-    // decision and this check will ensure we don't overlook it
-    final int oldestSupported = 8;
+    final int oldestSupported = AccumuloDataVersion.oldestUpgradeableVersion();
+    assertEquals(10, oldestSupported); // make sure it hasn't changed accidentally
     final int currentVersion = AccumuloDataVersion.get();
-    IntConsumer shouldPass = v -> ServerContext.ensureDataVersionCompatible(v);
+    IntConsumer shouldPass = ServerContext::ensureDataVersionCompatible;
     IntConsumer shouldFail = v -> assertThrows(IllegalStateException.class,
         () -> ServerContext.ensureDataVersionCompatible(v));
     IntStream.rangeClosed(oldestSupported, currentVersion).forEach(shouldPass);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
deleted file mode 100644
index 9b69efa..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ /dev/null
@@ -1,194 +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.server.client;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
-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.TableId;
-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.FileSKVWriter;
-import org.apache.accumulo.core.spi.crypto.CryptoService;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.server.MockServerContext;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-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.easymock.EasyMock;
-import org.junit.jupiter.api.Test;
-
-public class BulkImporterTest {
-
-  static final SortedSet<KeyExtent> fakeMetaData = new TreeSet<>();
-  static final TableId tableId = TableId.of("1");
-
-  static {
-    fakeMetaData.add(new KeyExtent(tableId, new Text("a"), null));
-    for (String part : new String[] {"b", "bm", "c", "cm", "d", "dm", "e", "em", "f", "g", "h", "i",
-        "j", "k", "l"}) {
-      fakeMetaData.add(new KeyExtent(tableId, new Text(part), fakeMetaData.last().endRow()));
-    }
-    fakeMetaData.add(new KeyExtent(tableId, null, fakeMetaData.last().endRow()));
-  }
-
-  class MockTabletLocator extends TabletLocator {
-    int invalidated = 0;
-
-    @Override
-    public TabletLocation locateTablet(ClientContext context, Text row, boolean skipRow,
-        boolean retry) {
-      return new TabletLocation(fakeMetaData.tailSet(new KeyExtent(tableId, row, null)).first(),
-          "localhost", "1");
-    }
-
-    @Override
-    public <T extends Mutation> void binMutations(ClientContext context, List<T> mutations,
-        Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public List<Range> binRanges(ClientContext context, List<Range> ranges,
-        Map<String,Map<KeyExtent,List<Range>>> binnedRanges) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void invalidateCache(KeyExtent failedExtent) {
-      invalidated++;
-    }
-
-    @Override
-    public void invalidateCache(Collection<KeyExtent> keySet) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void invalidateCache() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void invalidateCache(ClientContext context, String server) {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  @Test
-  public void testFindOverlappingTablets() throws Exception {
-    MockTabletLocator locator = new MockTabletLocator();
-    FileSystem fs = FileSystem.getLocal(new Configuration());
-    ServerContext context = MockServerContext.get();
-    CryptoService cs = NoCryptoServiceFactory.NONE;
-    EasyMock.replay(context);
-    String file = "target/testFile.rf";
-    fs.delete(new Path(file), true);
-    FileSKVWriter writer =
-        FileOperations.getInstance().newWriterBuilder().forFile(file, fs, fs.getConf(), cs)
-            .withTableConfiguration(context.getConfiguration()).build();
-    writer.startDefaultLocalityGroup();
-    Value empty = new Value();
-    writer.append(new Key("a", "cf", "cq"), empty);
-    writer.append(new Key("a", "cf", "cq1"), empty);
-    writer.append(new Key("a", "cf", "cq2"), empty);
-    writer.append(new Key("a", "cf", "cq3"), empty);
-    writer.append(new Key("a", "cf", "cq4"), empty);
-    writer.append(new Key("a", "cf", "cq5"), empty);
-    writer.append(new Key("d", "cf", "cq"), empty);
-    writer.append(new Key("d", "cf", "cq1"), empty);
-    writer.append(new Key("d", "cf", "cq2"), empty);
-    writer.append(new Key("d", "cf", "cq3"), empty);
-    writer.append(new Key("d", "cf", "cq4"), empty);
-    writer.append(new Key("d", "cf", "cq5"), empty);
-    writer.append(new Key("dd", "cf", "cq1"), empty);
-    writer.append(new Key("ichabod", "cf", "cq"), empty);
-    writer.append(new Key("icky", "cf", "cq1"), empty);
-    writer.append(new Key("iffy", "cf", "cq2"), empty);
-    writer.append(new Key("internal", "cf", "cq3"), empty);
-    writer.append(new Key("is", "cf", "cq4"), empty);
-    writer.append(new Key("iterator", "cf", "cq5"), empty);
-    writer.append(new Key("xyzzy", "cf", "cq"), empty);
-    writer.close();
-    try (var vm = VolumeManagerImpl.getLocalForTesting("file:///")) {
-      List<TabletLocation> overlaps =
-          BulkImporter.findOverlappingTablets(context, vm, locator, new Path(file), null, null, cs);
-      assertEquals(5, overlaps.size());
-      Collections.sort(overlaps);
-      assertEquals(new KeyExtent(tableId, new Text("a"), null), overlaps.get(0).tablet_extent);
-      assertEquals(new KeyExtent(tableId, new Text("d"), new Text("cm")),
-          overlaps.get(1).tablet_extent);
-      assertEquals(new KeyExtent(tableId, new Text("dm"), new Text("d")),
-          overlaps.get(2).tablet_extent);
-      assertEquals(new KeyExtent(tableId, new Text("j"), new Text("i")),
-          overlaps.get(3).tablet_extent);
-      assertEquals(new KeyExtent(tableId, null, new Text("l")), overlaps.get(4).tablet_extent);
-
-      List<TabletLocation> overlaps2 = BulkImporter.findOverlappingTablets(context, vm, locator,
-          new Path(file), new KeyExtent(tableId, new Text("h"), new Text("b")), cs);
-      assertEquals(3, overlaps2.size());
-      assertEquals(new KeyExtent(tableId, new Text("d"), new Text("cm")),
-          overlaps2.get(0).tablet_extent);
-      assertEquals(new KeyExtent(tableId, new Text("dm"), new Text("d")),
-          overlaps2.get(1).tablet_extent);
-      assertEquals(new KeyExtent(tableId, new Text("j"), new Text("i")),
-          overlaps2.get(2).tablet_extent);
-      assertEquals(locator.invalidated, 1);
-    }
-  }
-
-  @Test
-  public void testSequentialTablets() {
-    // ACCUMULO-3967 make sure that the startRow we compute in BulkImporter is actually giving
-    // a correct startRow so that findOverlappingTablets works as intended.
-
-    // 1;2;1
-    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("2"), new Text("1"));
-    assertEquals(new Text("1\0"), BulkImporter.getStartRowForExtent(extent));
-
-    // 1;2<
-    extent = new KeyExtent(TableId.of("1"), new Text("2"), null);
-    assertNull(BulkImporter.getStartRowForExtent(extent));
-
-    // 1<<
-    extent = new KeyExtent(TableId.of("1"), null, null);
-    assertNull(BulkImporter.getStartRowForExtent(extent));
-
-    // 1;8;7777777
-    extent = new KeyExtent(TableId.of("1"), new Text("8"), new Text("7777777"));
-    assertEquals(new Text("7777777\0"), BulkImporter.getStartRowForExtent(extent));
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/CheckCompactionConfigTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/CheckCompactionConfigTest.java
index c80ab28..ec90381 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/CheckCompactionConfigTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/CheckCompactionConfigTest.java
@@ -47,9 +47,9 @@
 
   @Test
   public void testValidInput1() throws Exception {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':4},\\\n"
         + "{'name':'large','type':'internal','numThreads':2}]").replaceAll("'", "\"");
@@ -61,15 +61,15 @@
 
   @Test
   public void testValidInput2() throws Exception {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':4},\\\n"
         + "{'name':'large','type':'internal','numThreads':2}] \n"
-        + "tserver.compaction.major.service.cs2.planner="
+        + "compaction.service.cs2.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs2.planner.opts.executors=\\\n"
+        + "compaction.service.cs2.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':7},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':5},\\\n"
         + "{'name':'large','type':'external','queue':'DCQ1'}]").replaceAll("'", "\"");
@@ -80,10 +80,33 @@
   }
 
   @Test
-  public void testThrowsExternalNumThreadsError() throws IOException {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+  public void testValidInput3() throws Exception {
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
+        + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
+        + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':4},\\\n"
+        + "{'name':'large','type':'internal','numThreads':2}] \n"
+        + "compaction.service.cs2.planner="
+        + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
+        + "compaction.service.cs2.planner.opts.executors=\\\n"
+        + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':7},\\\n"
+        + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':5},\\\n"
+        + "{'name':'large','type':'external','queue':'DCQ1'}] \n"
+        + "compaction.service.cs3.planner="
+        + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
+        + "compaction.service.cs3.planner.opts.queues=\\\n"
+        + "[{'name':'small','maxSize':'16M'},{'name':'large'}]").replaceAll("'", "\"");
+
+    String filePath = writeToFileAndReturnPath(inputString);
+    CheckCompactionConfig.main(new String[] {filePath});
+  }
+
+  @Test
+  public void testThrowsExternalNumThreadsError() throws IOException {
+    String inputString = ("compaction.service.cs1.planner="
+        + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'external','maxSize':'128M','numThreads':4},\\\n"
         + "{'name':'large','type':'internal','numThreads':2}]").replaceAll("'", "\"");
@@ -93,14 +116,14 @@
 
     var e = assertThrows(IllegalArgumentException.class,
         () -> CheckCompactionConfig.main(new String[] {filePath}));
-    assertEquals(e.getMessage(), expectedErrorMsg);
+    assertEquals(expectedErrorMsg, e.getMessage());
   }
 
   @Test
   public void testNegativeThreadCount() throws IOException {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':-4},\\\n"
         + "{'name':'large','type':'internal','numThreads':2}]").replaceAll("'", "\"");
@@ -115,11 +138,11 @@
 
   @Test
   public void testNoPlanner() throws Exception {
-    String inputString = ("tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+    String inputString = ("compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':4},\\\n"
         + "{'name':'large','type':'internal','numThreads':2}]").replaceAll("'", "\"");
-    String expectedErrorMsg = "Incomplete compaction service definitions, missing planner class";
+    String expectedErrorMsg = "Incomplete compaction service definition, missing planner class";
 
     String filePath = writeToFileAndReturnPath(inputString);
 
@@ -130,15 +153,15 @@
 
   @Test
   public void testRepeatedCompactionExecutorID() throws Exception {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':4},\\\n"
         + "{'name':'small','type':'internal','numThreads':2}]").replaceAll("'", "\"");
     String expectedErrorMsg = "Duplicate Compaction Executor ID found";
 
-    String filePath = writeToFileAndReturnPath(inputString);
+    final String filePath = writeToFileAndReturnPath(inputString);
 
     var e = assertThrows(IllegalStateException.class,
         () -> CheckCompactionConfig.main(new String[] {filePath}));
@@ -146,10 +169,28 @@
   }
 
   @Test
-  public void testInvalidTypeValue() throws Exception {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+  public void testRepeatedQueueName() throws Exception {
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
+        + "[{'name':'small','type':'external','maxSize':'16M','queue':'failedQueue'}] \n"
+        + "compaction.service.cs1.planner.opts.queues=[{'name':'failedQueue'}]")
+        .replaceAll("'", "\"");
+
+    String expectedErrorMsg = "Duplicate external executor for queue failedQueue";
+
+    final String filePath = writeToFileAndReturnPath(inputString);
+
+    var err = assertThrows(IllegalArgumentException.class,
+        () -> CheckCompactionConfig.main(new String[] {filePath}));
+    assertEquals(err.getMessage(), expectedErrorMsg);
+  }
+
+  @Test
+  public void testInvalidTypeValue() throws Exception {
+    String inputString = ("compaction.service.cs1.planner="
+        + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':4},\\\n"
         + "{'name':'large','type':'internl','numThreads':2}]").replaceAll("'", "\"");
@@ -164,9 +205,9 @@
 
   @Test
   public void testInvalidMaxSize() throws Exception {
-    String inputString = ("tserver.compaction.major.service.cs1.planner="
+    String inputString = ("compaction.service.cs1.planner="
         + "org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner \n"
-        + "tserver.compaction.major.service.cs1.planner.opts.executors=\\\n"
+        + "compaction.service.cs1.planner.opts.executors=\\\n"
         + "[{'name':'small','type':'internal','maxSize':'16M','numThreads':8},\\\n"
         + "{'name':'medium','type':'internal','maxSize':'0M','numThreads':4},\\\n"
         + "{'name':'large','type':'internal','numThreads':2}]").replaceAll("'", "\"");
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooBasedConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooBasedConfigurationTest.java
index 8859dd7..78c5f70 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooBasedConfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooBasedConfigurationTest.java
@@ -18,8 +18,9 @@
  */
 package org.apache.accumulo.server.conf;
 
+import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.accumulo.core.conf.Property.GC_PORT;
-import static org.apache.accumulo.core.conf.Property.MANAGER_BULK_RETRIES;
+import static org.apache.accumulo.core.conf.Property.MANAGER_BULK_TIMEOUT;
 import static org.apache.accumulo.core.conf.Property.TABLE_BLOOM_ENABLED;
 import static org.apache.accumulo.core.conf.Property.TABLE_BLOOM_SIZE;
 import static org.apache.accumulo.core.conf.Property.TABLE_DURABILITY;
@@ -167,7 +168,7 @@
     assertEquals("9998", zbc.get(GC_PORT));
 
     // read a property from the sysconfig
-    assertEquals("3", zbc.get(MANAGER_BULK_RETRIES));
+    assertEquals(MINUTES.toMillis(5), zbc.getTimeInMillis(MANAGER_BULK_TIMEOUT));
   }
 
   @Test
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/codec/VersionedPropEncryptCodec.java b/server/base/src/test/java/org/apache/accumulo/server/conf/codec/VersionedPropEncryptCodec.java
index cc6aedd..b76dae2 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/codec/VersionedPropEncryptCodec.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/codec/VersionedPropEncryptCodec.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.server.conf.codec;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -27,7 +29,6 @@
 import java.security.InvalidAlgorithmParameterException;
 import java.security.InvalidKeyException;
 import java.security.NoSuchAlgorithmException;
-import java.security.SecureRandom;
 import java.security.spec.InvalidKeySpecException;
 import java.security.spec.KeySpec;
 import java.util.Map;
@@ -54,8 +55,6 @@
  */
 public class VersionedPropEncryptCodec extends VersionedPropCodec {
 
-  private static final SecureRandom random = new SecureRandom();
-
   // testing version (999 or higher)
   public static final int EXPERIMENTAL_CIPHER_ENCODING_1_0 = 999;
 
@@ -217,7 +216,7 @@
     // utils
     public static GCMParameterSpec buildGCMParameterSpec() {
       byte[] iv = new byte[16];
-      random.nextBytes(iv);
+      RANDOM.get().nextBytes(iv);
       return new GCMParameterSpec(128, iv);
     }
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgraderTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgraderTest.java
deleted file mode 100644
index 929cdf0..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ConfigPropertyUpgraderTest.java
+++ /dev/null
@@ -1,102 +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.server.conf.util;
-
-import static org.easymock.EasyMock.anyString;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.NamespaceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.NamespacePropKey;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.store.TablePropKey;
-import org.junit.jupiter.api.Test;
-
-class ConfigPropertyUpgraderTest {
-
-  @Test
-  void upgradeSysProps() {
-
-    InstanceId iid = InstanceId.of(UUID.randomUUID());
-
-    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
-    ConfigTransformer transformer = createMock(ConfigTransformer.class);
-    expect(transformer.transform(SystemPropKey.of(iid), SystemPropKey.of(iid).getPath(), false))
-        .andReturn(new VersionedProperties()).once();
-
-    replay(transformer);
-    upgrader.upgradeSysProps(iid, transformer);
-    verify(transformer);
-  }
-
-  @Test
-  void upgradeNamespaceProps() throws Exception {
-
-    InstanceId iid = InstanceId.of(UUID.randomUUID());
-
-    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
-
-    ConfigTransformer transformer = createMock(ConfigTransformer.class);
-    String nsRoot = ZooUtil.getRoot(iid) + Constants.ZNAMESPACES;
-    expect(transformer.transform(NamespacePropKey.of(iid, NamespaceId.of("a")),
-        nsRoot + "/a" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
-    expect(transformer.transform(NamespacePropKey.of(iid, NamespaceId.of("b")),
-        nsRoot + "/b" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
-
-    ZooReaderWriter zrw = createMock(ZooReaderWriter.class);
-    expect(zrw.getChildren(anyString())).andReturn(List.of("a", "b")).once();
-
-    replay(transformer, zrw);
-    upgrader.upgradeNamespaceProps(iid, zrw, transformer);
-    verify(transformer, zrw);
-  }
-
-  @Test
-  void upgradeTableProps() throws Exception {
-
-    InstanceId iid = InstanceId.of(UUID.randomUUID());
-
-    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
-
-    ConfigTransformer transformer = createMock(ConfigTransformer.class);
-    String nsRoot = ZooUtil.getRoot(iid) + Constants.ZTABLES;
-    expect(transformer.transform(TablePropKey.of(iid, TableId.of("a")),
-        nsRoot + "/a" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
-    expect(transformer.transform(TablePropKey.of(iid, TableId.of("b")),
-        nsRoot + "/b" + Constants.ZCONF_LEGACY, true)).andReturn(new VersionedProperties()).once();
-
-    ZooReaderWriter zrw = createMock(ZooReaderWriter.class);
-    expect(zrw.getChildren(anyString())).andReturn(List.of("a", "b")).once();
-
-    replay(transformer, zrw);
-    upgrader.upgradeTableProps(iid, zrw, transformer);
-    verify(transformer, zrw);
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java
index bd59fef..b209a01 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java
@@ -22,17 +22,25 @@
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 
+import java.lang.reflect.Method;
+import java.util.Base64;
 import java.util.List;
 
+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.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+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.TabletColumnFamily;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.Test;
@@ -104,7 +112,7 @@
 
     assertNull(violations);
 
-    m = new Mutation(new Text("!0<"));
+    m = new Mutation(new Text(AccumuloTable.METADATA.tableId().canonical() + "<"));
     TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("bar"));
 
     violations = mc.check(createEnv(), m);
@@ -130,75 +138,330 @@
 
     // loaded marker w/ file
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
-    m.put(DataFileColumnFamily.NAME, new Text("/someFile"),
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
+    m.put(
+        DataFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
         new DataFileValue(1, 1).encodeAsValue());
     violations = mc.check(createEnv(), m);
     assertNull(violations);
 
     // loaded marker w/o file
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
-    violations = mc.check(createEnv(), m);
-    assertNotNull(violations);
-    assertEquals(1, violations.size());
-    assertEquals(Short.valueOf((short) 8), violations.get(0));
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
+    assertViolation(mc, m, (short) 8);
 
     // two files w/ same txid
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
-    m.put(DataFileColumnFamily.NAME, new Text("/someFile"),
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
+    m.put(
+        DataFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
         new DataFileValue(1, 1).encodeAsValue());
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile2"), new Value("5"));
-    m.put(DataFileColumnFamily.NAME, new Text("/someFile2"),
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile2")).getMetadataText(),
+        new Value("5"));
+    m.put(
+        DataFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile2")).getMetadataText(),
         new DataFileValue(1, 1).encodeAsValue());
     violations = mc.check(createEnv(), m);
     assertNull(violations);
 
     // two files w/ different txid
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
-    m.put(DataFileColumnFamily.NAME, new Text("/someFile"),
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
+    m.put(
+        DataFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
         new DataFileValue(1, 1).encodeAsValue());
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile2"), new Value("7"));
-    m.put(DataFileColumnFamily.NAME, new Text("/someFile2"),
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile2")).getMetadataText(),
+        new Value("7"));
+    m.put(
+        DataFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile2")).getMetadataText(),
         new DataFileValue(1, 1).encodeAsValue());
-    violations = mc.check(createEnv(), m);
-    assertNotNull(violations);
-    assertEquals(1, violations.size());
-    assertEquals(Short.valueOf((short) 8), violations.get(0));
+    assertViolation(mc, m, (short) 8);
 
     // two loaded markers but only one file.
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
-    m.put(DataFileColumnFamily.NAME, new Text("/someFile"),
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
+    m.put(
+        DataFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
         new DataFileValue(1, 1).encodeAsValue());
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile2"), new Value("5"));
-    violations = mc.check(createEnv(), m);
-    assertNotNull(violations);
-    assertEquals(1, violations.size());
-    assertEquals(Short.valueOf((short) 8), violations.get(0));
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile2")).getMetadataText(),
+        new Value("5"));
+    assertViolation(mc, m, (short) 8);
 
     // mutation that looks like split
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
     ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value("/t1"));
     violations = mc.check(createEnv(), m);
     assertNull(violations);
 
     // mutation that looks like a load
     m = new Mutation(new Text("0;foo"));
-    m.put(BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5"));
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
     m.put(CurrentLocationColumnFamily.NAME, new Text("789"), new Value("127.0.0.1:9997"));
     violations = mc.check(createEnv(), m);
     assertNull(violations);
 
     // deleting a load flag
     m = new Mutation(new Text("0;foo"));
-    m.putDelete(BulkFileColumnFamily.NAME, new Text("/someFile"));
+    m.putDelete(BulkFileColumnFamily.NAME, StoredTabletFile
+        .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText());
     violations = mc.check(createEnv(), m);
     assertNull(violations);
 
+    // Missing beginning of path
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME,
+        new Text(StoredTabletFile.of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"))
+            .getMetadata()
+            .replace("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile", "/someFile")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Missing tables directory in path
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME,
+        new Text(StoredTabletFile.of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"))
+            .getMetadata().replace("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile",
+                "hdfs://1.2.3.4/accumulo/2a/t-0003/someFile")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // No DataFileColumnFamily included
+    m = new Mutation(new Text("0;foo"));
+    m.put(
+        BulkFileColumnFamily.NAME, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new Value("5"));
+    assertViolation(mc, m, (short) 8);
+
+    // Bad Json - only path (old format) so should fail parsing
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME, new Text("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test startRow key is missing so validation should fail
+    // {"path":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME,
+        new Text(
+            "{\"path\":\"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile\",\"endRow\":\"\"}"),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test path key replaced with empty string so validation should fail
+    // {"":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","startRow":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(
+        BulkFileColumnFamily.NAME, new Text(StoredTabletFile
+            .serialize("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile").replace("path", "")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test path value missing
+    // {"path":"","startRow":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME,
+        new Text(StoredTabletFile.of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"))
+            .getMetadata().replaceFirst("\"path\":\".*\",\"startRow", "\"path\":\"\",\"startRow")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test startRow key replaced with empty string so validation should fail
+    // {"path":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME, new Text(StoredTabletFile
+        .serialize("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile").replace("startRow", "")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test endRow key missing so validation should fail
+    m = new Mutation(new Text("0;foo"));
+    m.put(
+        BulkFileColumnFamily.NAME, new Text(StoredTabletFile
+            .serialize("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile").replace("endRow", "")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - endRow will be replaced with encoded row without the exclusive byte 0x00 which is
+    // required for an endRow so will fail validation
+    m = new Mutation(new Text("0;foo"));
+    m.put(BulkFileColumnFamily.NAME,
+        new Text(StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"), new Range("a", "b"))
+            .getMetadata().replaceFirst("\"endRow\":\".*\"",
+                "\"endRow\":\"" + encodeRowForMetadata("bad") + "\"")),
+        new Value("5"));
+    assertViolation(mc, m, (short) 9);
+
+  }
+
+  @Test
+  public void testDataFileCheck() {
+    testFileMetadataValidation(DataFileColumnFamily.NAME, new DataFileValue(1, 1).encodeAsValue());
+  }
+
+  @Test
+  public void testScanFileCheck() {
+    testFileMetadataValidation(ScanFileColumnFamily.NAME, new Value());
+  }
+
+  private void testFileMetadataValidation(Text columnFamily, Value value) {
+    MetadataConstraints mc = new MetadataConstraints();
+    Mutation m;
+    List<Short> violations;
+
+    // Missing beginning of path
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily,
+        new Text(StoredTabletFile.of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"))
+            .getMetadata()
+            .replace("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile", "/someFile")),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - only path (old format) so should fail parsing
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily, new Text("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"), value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test path key replaced with empty string so validation should fail
+    // {"":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","startRow":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(
+        columnFamily, new Text(StoredTabletFile
+            .serialize("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile").replace("path", "")),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test path value missing
+    // {"path":"","startRow":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily,
+        new Text(StoredTabletFile.of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"))
+            .getMetadata().replaceFirst("\"path\":\".*\",\"startRow", "\"path\":\"\",\"startRow")),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test startRow key replaced with empty string so validation should fail
+    // {"path":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily, new Text(StoredTabletFile
+        .serialize("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile").replace("startRow", "")),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test startRow key is missing so validation should fail
+    // {"path":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily,
+        new Text(
+            "{\"path\":\"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile\",\"endRow\":\"\"}"),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - test endRow key replaced with empty string so validation should fail
+    // {"path":"hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile","":"","endRow":""}
+    m = new Mutation(new Text("0;foo"));
+    m.put(
+        columnFamily, new Text(StoredTabletFile
+            .serialize("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile").replace("endRow", "")),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Bad Json - endRow will be replaced with encoded row without the exclusive byte 0x00 which is
+    // required for an endRow so this will fail validation
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily,
+        new Text(StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"), new Range("a", "b"))
+            .getMetadata()
+            .replaceFirst("\"endRow\":\".*\"", "\"endRow\":\"" + encodeRowForMetadata("b") + "\"")),
+        value);
+    assertViolation(mc, m, (short) 9);
+
+    // Missing tables directory in path
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily,
+        new Text(StoredTabletFile.of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"))
+            .getMetadata().replace("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile",
+                "hdfs://1.2.3.4/accumulo/2a/t-0003/someFile")),
+        new DataFileValue(1, 1).encodeAsValue());
+    assertViolation(mc, m, (short) 9);
+
+    // Should pass validation (inf range)
+    m = new Mutation(new Text("0;foo"));
+    m.put(
+        columnFamily, StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile")).getMetadataText(),
+        new DataFileValue(1, 1).encodeAsValue());
+    violations = mc.check(createEnv(), m);
+    assertNull(violations);
+
+    // Should pass validation with range set
+    m = new Mutation(new Text("0;foo"));
+    m.put(columnFamily,
+        StoredTabletFile
+            .of(new Path("hdfs://1.2.3.4/accumulo/tables/2a/t-0003/someFile"), new Range("a", "b"))
+            .getMetadataText(),
+        new DataFileValue(1, 1).encodeAsValue());
+    violations = mc.check(createEnv(), m);
+    assertNull(violations);
+
+    assertNotNull(mc.getViolationDescription((short) 9));
+  }
+
+  // Encode a row how it would appear in Json
+  private static String encodeRowForMetadata(String row) {
+    try {
+      Method method = StoredTabletFile.class.getDeclaredMethod("encodeRow", Key.class);
+      method.setAccessible(true);
+      return Base64.getUrlEncoder()
+          .encodeToString((byte[]) method.invoke(StoredTabletFile.class, new Key(row)));
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private void assertViolation(MetadataConstraints mc, Mutation m, Short violation) {
+    List<Short> violations = mc.check(createEnv(), m);
+    assertNotNull(violations);
+    assertEquals(1, violations.size());
+    assertEquals(violation, violations.get(0));
+    assertNotNull(mc.getViolationDescription(violations.get(0)));
   }
 
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java
index 6770966..033cf17 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java
@@ -21,17 +21,13 @@
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.LinkedHashMap;
+import java.util.Map;
 import java.util.UUID;
 
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
@@ -41,157 +37,151 @@
 
   @Test
   public void testSwitchVolume() {
-    List<Pair<Path,Path>> replacements = new ArrayList<>();
-    replacements.add(new Pair<>(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/a/accumulo")));
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a/accumulo")));
-    replacements.add(new Pair<>(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b/accumulo")));
+    Map<Path,Path> replacements = new LinkedHashMap<>();
+    replacements.put(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/a/accumulo"));
+    replacements.put(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a/accumulo"));
+    replacements.put(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b/accumulo"));
 
-    assertEquals(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"), VolumeUtil
-        .switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
     assertEquals(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertEquals(new Path("viewfs:/b/accumulo/tables/t-00000/C000.rf"), VolumeUtil
-        .switchVolume("hdfs://nn2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("viewfs:/a/accumulo/tables/t-00000/C000.rf", FileType.TABLE,
-        replacements));
-    assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf",
+        new Path("hdfs://nn1/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
+        new Path("hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/b/accumulo/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
+        new Path("hdfs://nn2/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("file:/nn1/a/accumulo/tables/t-00000/C000.rf"),
         FileType.TABLE, replacements));
 
     replacements.clear();
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1/d1/accumulo"), new Path("viewfs:/a/accumulo")));
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1:9000/d1/accumulo"), new Path("viewfs:/a/accumulo")));
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn2/d2/accumulo"), new Path("viewfs:/b/accumulo")));
+    replacements.put(new Path("hdfs://nn1/d1/accumulo"), new Path("viewfs:/a/accumulo"));
+    replacements.put(new Path("hdfs://nn1:9000/d1/accumulo"), new Path("viewfs:/a/accumulo"));
+    replacements.put(new Path("hdfs://nn2/d2/accumulo"), new Path("viewfs:/b/accumulo"));
 
     assertEquals(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1/d1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertEquals(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1:9000/d1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
+        new Path("hdfs://nn1/d1/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"),
+        VolumeUtil.switchVolume(new Path("hdfs://nn1:9000/d1/accumulo/tables/t-00000/C000.rf"),
+            FileType.TABLE, replacements));
     assertEquals(new Path("viewfs:/b/accumulo/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn2/d2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("viewfs:/a/accumulo/tables/t-00000/C000.rf", FileType.TABLE,
-        replacements));
-    assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf",
+        new Path("hdfs://nn2/d2/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("viewfs:/a/accumulo/tables/t-00000/C000.rf"),
         FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE,
-        replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("file:/nn1/a/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("hdfs://nn1/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
   }
 
   @Test
   public void testSwitchVolumesDifferentSourceDepths() {
-    List<Pair<Path,Path>> replacements = new ArrayList<>();
-    replacements.add(new Pair<>(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/a")));
-    replacements.add(new Pair<>(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a")));
-    replacements.add(new Pair<>(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b")));
+    Map<Path,Path> replacements = new LinkedHashMap<>();
+    replacements.put(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/a"));
+    replacements.put(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a"));
+    replacements.put(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b"));
 
-    assertEquals(new Path("viewfs:/a/tables/t-00000/C000.rf"), VolumeUtil
-        .switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
     assertEquals(new Path("viewfs:/a/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertEquals(new Path("viewfs:/b/tables/t-00000/C000.rf"), VolumeUtil
-        .switchVolume("hdfs://nn2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(
-        VolumeUtil.switchVolume("viewfs:/a/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf",
+        new Path("hdfs://nn1/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/a/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
+        new Path("hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/b/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
+        new Path("hdfs://nn2/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("viewfs:/a/tables/t-00000/C000.rf"), FileType.TABLE,
+        replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("file:/nn1/a/accumulo/tables/t-00000/C000.rf"),
         FileType.TABLE, replacements));
 
     replacements.clear();
-    replacements.add(new Pair<>(new Path("hdfs://nn1/d1/accumulo"), new Path("viewfs:/a")));
-    replacements.add(new Pair<>(new Path("hdfs://nn1:9000/d1/accumulo"), new Path("viewfs:/a")));
-    replacements.add(new Pair<>(new Path("hdfs://nn2/d2/accumulo"), new Path("viewfs:/b")));
+    replacements.put(new Path("hdfs://nn1/d1/accumulo"), new Path("viewfs:/a"));
+    replacements.put(new Path("hdfs://nn1:9000/d1/accumulo"), new Path("viewfs:/a"));
+    replacements.put(new Path("hdfs://nn2/d2/accumulo"), new Path("viewfs:/b"));
 
     assertEquals(new Path("viewfs:/a/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1/d1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertEquals(new Path("viewfs:/a/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1:9000/d1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
+        new Path("hdfs://nn1/d1/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/a/tables/t-00000/C000.rf"),
+        VolumeUtil.switchVolume(new Path("hdfs://nn1:9000/d1/accumulo/tables/t-00000/C000.rf"),
+            FileType.TABLE, replacements));
     assertEquals(new Path("viewfs:/b/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn2/d2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(
-        VolumeUtil.switchVolume("viewfs:/a/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf",
-        FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE,
+        new Path("hdfs://nn2/d2/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("viewfs:/a/tables/t-00000/C000.rf"), FileType.TABLE,
         replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("file:/nn1/a/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("hdfs://nn1/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
   }
 
   @Test
   public void testSwitchVolumesDifferentTargetDepths() {
-    List<Pair<Path,Path>> replacements = new ArrayList<>();
-    replacements.add(new Pair<>(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/path1/path2")));
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/path1/path2")));
-    replacements.add(new Pair<>(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/path3")));
+    Map<Path,Path> replacements = new LinkedHashMap<>();
+    replacements.put(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/path1/path2"));
+    replacements.put(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/path1/path2"));
+    replacements.put(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/path3"));
 
-    assertEquals(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"), VolumeUtil
-        .switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
     assertEquals(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertEquals(new Path("viewfs:/path3/tables/t-00000/C000.rf"), VolumeUtil
-        .switchVolume("hdfs://nn2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("viewfs:/path1/path2/tables/t-00000/C000.rf", FileType.TABLE,
-        replacements));
-    assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf",
+        new Path("hdfs://nn1/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
+        new Path("hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/path3/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
+        new Path("hdfs://nn2/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("file:/nn1/a/accumulo/tables/t-00000/C000.rf"),
         FileType.TABLE, replacements));
 
     replacements.clear();
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1/d1/accumulo"), new Path("viewfs:/path1/path2")));
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1:9000/d1/accumulo"), new Path("viewfs:/path1/path2")));
-    replacements.add(new Pair<>(new Path("hdfs://nn2/d2/accumulo"), new Path("viewfs:/path3")));
+    replacements.put(new Path("hdfs://nn1/d1/accumulo"), new Path("viewfs:/path1/path2"));
+    replacements.put(new Path("hdfs://nn1:9000/d1/accumulo"), new Path("viewfs:/path1/path2"));
+    replacements.put(new Path("hdfs://nn2/d2/accumulo"), new Path("viewfs:/path3"));
 
     assertEquals(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1/d1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertEquals(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn1:9000/d1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
+        new Path("hdfs://nn1/d1/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertEquals(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"),
+        VolumeUtil.switchVolume(new Path("hdfs://nn1:9000/d1/accumulo/tables/t-00000/C000.rf"),
+            FileType.TABLE, replacements));
     assertEquals(new Path("viewfs:/path3/tables/t-00000/C000.rf"), VolumeUtil.switchVolume(
-        "hdfs://nn2/d2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("viewfs:/path1/path2/tables/t-00000/C000.rf", FileType.TABLE,
-        replacements));
-    assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf",
+        new Path("hdfs://nn2/d2/accumulo/tables/t-00000/C000.rf"), FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("viewfs:/path1/path2/tables/t-00000/C000.rf"),
         FileType.TABLE, replacements));
-    assertNull(VolumeUtil.switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE,
-        replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("file:/nn1/a/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
+    assertNull(VolumeUtil.switchVolume(new Path("hdfs://nn1/accumulo/tables/t-00000/C000.rf"),
+        FileType.TABLE, replacements));
   }
 
   @Test
   public void testRootTableReplacement() {
-    List<Pair<Path,Path>> replacements = new ArrayList<>();
-    replacements.add(new Pair<>(new Path("file:/foo/v1"), new Path("file:/foo/v8")));
-    replacements.add(new Pair<>(new Path("file:/foo/v2"), new Path("file:/foo/v9")));
+    Map<Path,Path> replacements = new LinkedHashMap<>();
+    replacements.put(new Path("file:/foo/v1"), new Path("file:/foo/v8"));
+    replacements.put(new Path("file:/foo/v2"), new Path("file:/foo/v9"));
 
     FileType ft = FileType.TABLE;
 
     assertEquals(new Path("file:/foo/v8/tables/+r/root_tablet"),
-        VolumeUtil.switchVolume("file:/foo/v1/tables/+r/root_tablet", ft, replacements));
+        VolumeUtil.switchVolume(new Path("file:/foo/v1/tables/+r/root_tablet"), ft, replacements));
   }
 
   @Test
   public void testWalVolumeReplacment() {
-    List<Pair<Path,Path>> replacements = new ArrayList<>();
-    replacements.add(new Pair<>(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/a/accumulo")));
-    replacements
-        .add(new Pair<>(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a/accumulo")));
-    replacements.add(new Pair<>(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b/accumulo")));
+    Map<Path,Path> replacements = new LinkedHashMap<>();
+    replacements.put(new Path("hdfs://nn1/accumulo"), new Path("viewfs:/a/accumulo"));
+    replacements.put(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a/accumulo"));
+    replacements.put(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b/accumulo"));
 
     String walUUID = UUID.randomUUID().toString();
-    KeyExtent ke = new KeyExtent(TableId.of("1"), new Text("z"), new Text("a"));
     String fileName = "hdfs://nn1/accumulo/wal/localhost+9997/" + walUUID;
-    LogEntry le = new LogEntry(ke, 1L, fileName);
-    LogEntry fixedVolume = VolumeUtil.switchVolumes(le, replacements);
-    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.filename);
+    LogEntry le = LogEntry.fromPath(fileName);
+    LogEntry fixedVolume = VolumeUtil.switchVolume(le, replacements);
+    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.getPath());
 
     fileName = "hdfs://nn1:9000/accumulo/wal/localhost+9997/" + walUUID;
-    le = new LogEntry(ke, 1L, fileName);
-    fixedVolume = VolumeUtil.switchVolumes(le, replacements);
-    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.filename);
+    le = LogEntry.fromPath(fileName);
+    fixedVolume = VolumeUtil.switchVolume(le, replacements);
+    assertEquals("viewfs:/a/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.getPath());
 
     fileName = "hdfs://nn2/accumulo/wal/localhost+9997/" + walUUID;
-    le = new LogEntry(ke, 1L, fileName);
-    fixedVolume = VolumeUtil.switchVolumes(le, replacements);
-    assertEquals("viewfs:/b/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.filename);
+    le = LogEntry.fromPath(fileName);
+    fixedVolume = VolumeUtil.switchVolume(le, replacements);
+    assertEquals("viewfs:/b/accumulo/wal/localhost+9997/" + walUUID, fixedVolume.getPath());
   }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
index 8ce7e55..1bffe13 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
@@ -51,7 +51,7 @@
     @Override
     public boolean transactionComplete(String type, long tid) {
       if (tid == 9) {
-        throw new RuntimeException();
+        throw new IllegalArgumentException();
       }
       return tid != 5 && tid != 7;
     }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java
index e767f3b..f67b7cb 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/manager/LiveTServerSetTest.java
@@ -25,7 +25,6 @@
 import java.util.Map;
 
 import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.manager.LiveTServerSet.Listener;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
@@ -33,6 +32,8 @@
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class LiveTServerSetTest {
 
   @Test
diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java
index f671238..01ae58b 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java
@@ -101,18 +101,6 @@
   }
 
   @Test
-  public void testNeedsToBeChopped_NotDelete() {
-    expect(keyExtent.tableId()).andReturn(TableId.of("table1"));
-    KeyExtent keyExtent2 = createMock(KeyExtent.class);
-    expect(keyExtent2.tableId()).andReturn(TableId.of("table1"));
-    replay(keyExtent2);
-    expect(keyExtent.overlaps(keyExtent2)).andReturn(true);
-    replay(keyExtent);
-    mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
-    assertTrue(mi.needsToBeChopped(keyExtent2));
-  }
-
-  @Test
   public void testNeedsToBeChopped_Delete_NotFollowing() {
     testNeedsToBeChopped_Delete("somerow", false);
   }
@@ -213,14 +201,6 @@
     assertFalse(info.needsToBeChopped(ke("y", "c", "b")));
     assertFalse(info.needsToBeChopped(ke("x", "c", "bb")));
     assertFalse(info.needsToBeChopped(ke("x", "b", "a")));
-    info = new MergeInfo(ke("x", "b", "a"), MergeInfo.Operation.MERGE);
-    assertTrue(info.needsToBeChopped(ke("x", "c", "a")));
-    assertTrue(info.needsToBeChopped(ke("x", "aa", "a")));
-    assertTrue(info.needsToBeChopped(ke("x", null, null)));
-    assertFalse(info.needsToBeChopped(ke("x", "c", "b")));
-    assertFalse(info.needsToBeChopped(ke("y", "c", "b")));
-    assertFalse(info.needsToBeChopped(ke("x", "c", "bb")));
-    assertTrue(info.needsToBeChopped(ke("x", "b", "a")));
   }
 
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java
index 68dd6c5..3e3241e 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/RootTabletStateStoreTest.java
@@ -43,7 +43,6 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.server.MockServerContext;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.metadata.TabletMutatorBase;
@@ -51,6 +50,7 @@
 import org.junit.jupiter.api.Test;
 
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 public class RootTabletStateStoreTest {
 
@@ -120,8 +120,7 @@
     assertEquals(count, 1);
     TabletLocationState assigned = null;
     try {
-      assigned =
-          new TabletLocationState(root, Location.future(server), null, null, null, null, false);
+      assigned = new TabletLocationState(root, Location.future(server), null, null, null, null);
     } catch (BadLocationStateException e) {
       fail("Unexpected error " + e);
     }
@@ -143,7 +142,7 @@
 
     try {
       TabletLocationState broken =
-          new TabletLocationState(notRoot, Location.future(server), null, null, null, null, false);
+          new TabletLocationState(notRoot, Location.future(server), null, null, null, null);
       final var assignmentList1 = List.of(broken);
       assertThrows(IllegalArgumentException.class, () -> tstore.unassign(assignmentList1, null));
     } catch (BadLocationStateException e) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletLocationStateTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletLocationStateTest.java
index c6e21bb..0933173 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletLocationStateTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/TabletLocationStateTest.java
@@ -26,30 +26,23 @@
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.util.Collection;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletState;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 public class TabletLocationStateTest {
-  private static final Collection<String> innerWalogs = new java.util.HashSet<>();
-  private static final Collection<Collection<String>> walogs = new java.util.HashSet<>();
-
-  @BeforeAll
-  public static void setUpClass() {
-    walogs.add(innerWalogs);
-    innerWalogs.add("somelog");
-  }
+  private static final Set<LogEntry> walogs =
+      Set.of(LogEntry.fromPath("file:///dir/tserver+9997/" + UUID.randomUUID()));
 
   private KeyExtent keyExtent;
   private Location future;
@@ -67,24 +60,22 @@
 
   @Test
   public void testConstruction_NoFuture() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs);
     assertSame(keyExtent, tls.extent);
     assertNull(tls.future);
     assertSame(current, tls.current);
     assertSame(last, tls.last);
     assertSame(walogs, tls.walogs);
-    assertTrue(tls.chopped);
   }
 
   @Test
   public void testConstruction_NoCurrent() throws Exception {
-    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs);
     assertSame(keyExtent, tls.extent);
     assertSame(future, tls.future);
     assertNull(tls.current);
     assertSame(last, tls.last);
     assertSame(walogs, tls.walogs);
-    assertTrue(tls.chopped);
   }
 
   @Test
@@ -92,50 +83,50 @@
     expect(keyExtent.toMetaRow()).andReturn(new Text("entry"));
     replay(keyExtent);
     var e = assertThrows(TabletLocationState.BadLocationStateException.class,
-        () -> new TabletLocationState(keyExtent, future, current, last, null, walogs, true));
+        () -> new TabletLocationState(keyExtent, future, current, last, null, walogs));
     assertEquals(new Text("entry"), e.getEncodedEndRow());
   }
 
   @Test
   public void testConstruction_NoFuture_NoWalogs() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, current, last, null, null, true);
+    tls = new TabletLocationState(keyExtent, null, current, last, null, null);
     assertNotNull(tls.walogs);
     assertEquals(0, tls.walogs.size());
   }
 
   @Test
   public void testGetServer_Current() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs);
     assertSame(current, tls.getLocation());
   }
 
   @Test
   public void testGetServer_Future() throws Exception {
-    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs);
     assertSame(future, tls.getLocation());
   }
 
   @Test
   public void testGetServer_Last() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, null, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, null, last, null, walogs);
     assertSame(last, tls.getLocation());
   }
 
   @Test
   public void testGetServer_None() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, null, null, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, null, null, null, walogs);
     assertNull(tls.getLocation());
   }
 
   @Test
   public void testGetState_Unassigned1() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, null, null, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, null, null, null, walogs);
     assertEquals(TabletState.UNASSIGNED, tls.getState(null));
   }
 
   @Test
   public void testGetState_Unassigned2() throws Exception {
-    tls = new TabletLocationState(keyExtent, null, null, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, null, last, null, walogs);
     assertEquals(TabletState.UNASSIGNED, tls.getState(null));
   }
 
@@ -143,7 +134,7 @@
   public void testGetState_Assigned() throws Exception {
     Set<TServerInstance> liveServers = new java.util.HashSet<>();
     liveServers.add(future.getServerInstance());
-    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs);
     assertEquals(TabletState.ASSIGNED, tls.getState(liveServers));
   }
 
@@ -151,7 +142,7 @@
   public void testGetState_Hosted() throws Exception {
     Set<TServerInstance> liveServers = new java.util.HashSet<>();
     liveServers.add(current.getServerInstance());
-    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs);
     assertEquals(TabletState.HOSTED, tls.getState(liveServers));
   }
 
@@ -159,7 +150,7 @@
   public void testGetState_Dead1() throws Exception {
     Set<TServerInstance> liveServers = new java.util.HashSet<>();
     liveServers.add(current.getServerInstance());
-    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, future, null, last, null, walogs);
     assertEquals(TabletState.ASSIGNED_TO_DEAD_SERVER, tls.getState(liveServers));
   }
 
@@ -167,7 +158,7 @@
   public void testGetState_Dead2() throws Exception {
     Set<TServerInstance> liveServers = new java.util.HashSet<>();
     liveServers.add(future.getServerInstance());
-    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs, true);
+    tls = new TabletLocationState(keyExtent, null, current, last, null, walogs);
     assertEquals(TabletState.ASSIGNED_TO_DEAD_SERVER, tls.getState(liveServers));
   }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
deleted file mode 100644
index 7d935d2..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ /dev/null
@@ -1,371 +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.server.master.balancer;
-
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.function.Predicate;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Namespace;
-import org.apache.accumulo.core.clientImpl.TableOperationsImpl;
-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.conf.SiteConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.NamespaceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.NamespaceConfiguration;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.conf.codec.VersionedProperties;
-import org.apache.accumulo.server.conf.store.NamespacePropKey;
-import org.apache.accumulo.server.conf.store.PropStore;
-import org.apache.accumulo.server.conf.store.TablePropKey;
-import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-
-@Deprecated(since = "2.1.0")
-public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableLoadBalancer {
-
-  protected static class TestTable {
-    private String tableName;
-    private TableId id;
-
-    TestTable(String tableName, TableId id) {
-      this.tableName = tableName;
-      this.id = id;
-    }
-
-    public String getTableName() {
-      return tableName;
-    }
-
-    public TableId getId() {
-      return id;
-    }
-  }
-
-  protected static final HashMap<String,String> DEFAULT_TABLE_PROPERTIES = new HashMap<>();
-  {
-    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "7s");
-    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY,
-        "4");
-    DEFAULT_TABLE_PROPERTIES
-        .put(HostRegexTableLoadBalancer.HOST_BALANCER_OUTSTANDING_MIGRATIONS_KEY, "10");
-    DEFAULT_TABLE_PROPERTIES
-        .put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(), "r01.*");
-    DEFAULT_TABLE_PROPERTIES
-        .put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r02.*");
-    DEFAULT_TABLE_PROPERTIES.put(Property.TABLE_LOAD_BALANCER.getKey(),
-        TestDefaultBalancer.class.getName());
-  }
-
-  private static SiteConfiguration siteConfg = SiteConfiguration.empty().build();
-
-  protected static class TestServerConfigurationFactory extends ServerConfigurationFactory {
-
-    final ServerContext context;
-    private ConfigurationCopy config;
-
-    public TestServerConfigurationFactory(ServerContext context) {
-      super(context, siteConfg);
-      this.context = context;
-      this.config = new ConfigurationCopy(DEFAULT_TABLE_PROPERTIES);
-    }
-
-    @Override
-    public synchronized AccumuloConfiguration getSystemConfiguration() {
-      return config;
-    }
-
-    @Override
-    public TableConfiguration getTableConfiguration(final TableId tableId) {
-      // create a dummy namespaceConfiguration to satisfy requireNonNull in TableConfiguration
-      // constructor
-      NamespaceConfiguration dummyConf = new NamespaceConfiguration(context, Namespace.DEFAULT.id(),
-          DefaultConfiguration.getInstance());
-      return new TableConfiguration(context, tableId, dummyConf) {
-        @Override
-        public String get(Property property) {
-          return getSystemConfiguration().get(property.name());
-        }
-
-        @Override
-        public void getProperties(Map<String,String> props, Predicate<String> filter) {
-          getSystemConfiguration().getProperties(props, filter);
-        }
-
-        @Override
-        public long getUpdateCount() {
-          return 0;
-        }
-      };
-    }
-  }
-
-  protected static final TestTable FOO = new TestTable("foo", TableId.of("1"));
-  protected static final TestTable BAR = new TestTable("bar", TableId.of("2"));
-  protected static final TestTable BAZ = new TestTable("baz", TableId.of("3"));
-
-  protected class TestDefaultBalancer extends DefaultLoadBalancer {
-    @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
-      String tableName = idToTableName(tableId);
-      TServerInstance initialLocation = initialTableLocation.get(tableName);
-      if (tserver.equals(initialLocation)) {
-        List<TabletStats> list = new ArrayList<>(5);
-        for (KeyExtent extent : tableExtents.get(tableName)) {
-          TabletStats stats = new TabletStats();
-          stats.setExtent(extent.toThrift());
-          list.add(stats);
-        }
-        return list;
-      }
-      return null;
-    }
-  }
-
-  protected ServerContext createMockContext() {
-    InstanceId instanceId = InstanceId.of(UUID.randomUUID());
-
-    ServerContext mockContext = createMock(ServerContext.class);
-    PropStore propStore = createMock(ZooPropStore.class);
-    expect(mockContext.getProperties()).andReturn(new Properties()).anyTimes();
-    expect(mockContext.getZooKeepers()).andReturn("").anyTimes();
-    expect(mockContext.getInstanceName()).andReturn("test").anyTimes();
-    expect(mockContext.getZooKeepersSessionTimeOut()).andReturn(30).anyTimes();
-    expect(mockContext.getInstanceID()).andReturn(instanceId).anyTimes();
-    expect(mockContext.getZooKeeperRoot()).andReturn(Constants.ZROOT + "/1111").anyTimes();
-
-    expect(mockContext.getPropStore()).andReturn(propStore).anyTimes();
-    propStore.registerAsListener(anyObject(), anyObject());
-    expectLastCall().anyTimes();
-
-    expect(propStore.get(eq(NamespacePropKey.of(instanceId, NamespaceId.of("+default")))))
-        .andReturn(new VersionedProperties()).anyTimes();
-
-    expect(propStore.get(eq(TablePropKey.of(instanceId, TableId.of("1")))))
-        .andReturn(new VersionedProperties()).anyTimes();
-
-    expect(propStore.get(eq(TablePropKey.of(instanceId, TableId.of("2")))))
-        .andReturn(new VersionedProperties()).anyTimes();
-
-    expect(propStore.get(eq(TablePropKey.of(instanceId, TableId.of("3")))))
-        .andReturn(new VersionedProperties()).anyTimes();
-
-    replay(propStore);
-    return mockContext;
-  }
-
-  protected final Map<String,String> servers = new HashMap<>(15);
-  protected final SortedMap<TServerInstance,TabletServerStatus> allTabletServers = new TreeMap<>();
-  protected final Map<String,List<KeyExtent>> tableExtents = new HashMap<>(3);
-  protected final Map<String,TServerInstance> initialTableLocation = new HashMap<>(3);
-
-  {
-    servers.put("192.168.0.1", "r01s01");
-    servers.put("192.168.0.2", "r01s02");
-    servers.put("192.168.0.3", "r01s03");
-    servers.put("192.168.0.4", "r01s04");
-    servers.put("192.168.0.5", "r01s05");
-    servers.put("192.168.0.6", "r02s01");
-    servers.put("192.168.0.7", "r02s02");
-    servers.put("192.168.0.8", "r02s03");
-    servers.put("192.168.0.9", "r02s04");
-    servers.put("192.168.0.10", "r02s05");
-    servers.put("192.168.0.11", "r03s01");
-    servers.put("192.168.0.12", "r03s02");
-    servers.put("192.168.0.13", "r03s03");
-    servers.put("192.168.0.14", "r03s04");
-    servers.put("192.168.0.15", "r03s05");
-
-    allTabletServers.put(new TServerInstance("192.168.0.1:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.2:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.3:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.4:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.5:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.6:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.7:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.8:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.9:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.10:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.11:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.12:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.13:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.14:9997", 1), new TabletServerStatus());
-    allTabletServers.put(new TServerInstance("192.168.0.15:9997", 1), new TabletServerStatus());
-
-    initialTableLocation.put(FOO.getTableName(), new TServerInstance("192.168.0.1:9997", 1));
-    initialTableLocation.put(BAR.getTableName(), new TServerInstance("192.168.0.6:9997", 1));
-    initialTableLocation.put(BAZ.getTableName(), new TServerInstance("192.168.0.11:9997", 1));
-
-    tableExtents.put(FOO.getTableName(), new ArrayList<>());
-    tableExtents.get(FOO.getTableName())
-        .add(new KeyExtent(FOO.getId(), new Text("1"), new Text("0")));
-    tableExtents.get(FOO.getTableName())
-        .add(new KeyExtent(FOO.getId(), new Text("2"), new Text("1")));
-    tableExtents.get(FOO.getTableName())
-        .add(new KeyExtent(FOO.getId(), new Text("3"), new Text("2")));
-    tableExtents.get(FOO.getTableName())
-        .add(new KeyExtent(FOO.getId(), new Text("4"), new Text("3")));
-    tableExtents.get(FOO.getTableName())
-        .add(new KeyExtent(FOO.getId(), new Text("5"), new Text("4")));
-    tableExtents.put(BAR.getTableName(), new ArrayList<>());
-    tableExtents.get(BAR.getTableName())
-        .add(new KeyExtent(BAR.getId(), new Text("11"), new Text("10")));
-    tableExtents.get(BAR.getTableName())
-        .add(new KeyExtent(BAR.getId(), new Text("12"), new Text("11")));
-    tableExtents.get(BAR.getTableName())
-        .add(new KeyExtent(BAR.getId(), new Text("13"), new Text("12")));
-    tableExtents.get(BAR.getTableName())
-        .add(new KeyExtent(BAR.getId(), new Text("14"), new Text("13")));
-    tableExtents.get(BAR.getTableName())
-        .add(new KeyExtent(BAR.getId(), new Text("15"), new Text("14")));
-    tableExtents.put(BAZ.getTableName(), new ArrayList<>());
-    tableExtents.get(BAZ.getTableName())
-        .add(new KeyExtent(BAZ.getId(), new Text("21"), new Text("20")));
-    tableExtents.get(BAZ.getTableName())
-        .add(new KeyExtent(BAZ.getId(), new Text("22"), new Text("21")));
-    tableExtents.get(BAZ.getTableName())
-        .add(new KeyExtent(BAZ.getId(), new Text("23"), new Text("22")));
-    tableExtents.get(BAZ.getTableName())
-        .add(new KeyExtent(BAZ.getId(), new Text("24"), new Text("23")));
-    tableExtents.get(BAZ.getTableName())
-        .add(new KeyExtent(BAZ.getId(), new Text("25"), new Text("24")));
-
-  }
-
-  protected boolean tabletInBounds(KeyExtent ke, TServerInstance tsi) {
-    String tid = ke.tableId().canonical();
-    String host = tsi.getHost();
-    if (tid.equals("1")
-        && (host.equals("192.168.0.1") || host.equals("192.168.0.2") || host.equals("192.168.0.3")
-            || host.equals("192.168.0.4") || host.equals("192.168.0.5"))) {
-      return true;
-    } else if (tid.equals("2")
-        && (host.equals("192.168.0.6") || host.equals("192.168.0.7") || host.equals("192.168.0.8")
-            || host.equals("192.168.0.9") || host.equals("192.168.0.10"))) {
-      return true;
-    } else {
-      return tid.equals("3") && (host.equals("192.168.0.11") || host.equals("192.168.0.12")
-          || host.equals("192.168.0.13") || host.equals("192.168.0.14")
-          || host.equals("192.168.0.15"));
-    }
-  }
-
-  protected String idToTableName(TableId id) {
-    if (id.equals(FOO.getId())) {
-      return FOO.getTableName();
-    } else if (id.equals(BAR.getId())) {
-      return BAR.getTableName();
-    } else if (id.equals(BAZ.getId())) {
-      return BAZ.getTableName();
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  protected TableOperations getTableOperations() {
-    return new TableOperationsImpl(EasyMock.createMock(ClientContext.class)) {
-      @Override
-      public Map<String,String> tableIdMap() {
-        HashMap<String,String> tables = new HashMap<>();
-        tables.put(FOO.getTableName(), FOO.getId().canonical());
-        tables.put(BAR.getTableName(), BAR.getId().canonical());
-        tables.put(BAZ.getTableName(), BAZ.getId().canonical());
-        return tables;
-      }
-
-      @Override
-      public SortedSet<String> list() {
-        TreeSet<String> tables = new TreeSet<>();
-        tables.add(BAR.getTableName());
-        tables.add(BAZ.getTableName());
-        tables.add(FOO.getTableName());
-        return tables;
-      }
-    };
-  }
-
-  @Override
-  protected TabletBalancer getBalancerForTable(TableId table) {
-    return new TestDefaultBalancer();
-  }
-
-  @Override
-  protected String getNameFromIp(String hostIp) throws UnknownHostException {
-    if (servers.containsKey(hostIp)) {
-      return servers.get(hostIp);
-    } else {
-      throw new UnknownHostException();
-    }
-  }
-
-  protected SortedMap<TServerInstance,TabletServerStatus> createCurrent(int numTservers) {
-    String base = "192.168.0.";
-    TreeMap<TServerInstance,TabletServerStatus> current = new TreeMap<>();
-    for (int i = 1; i <= numTservers; i++) {
-      TabletServerStatus status = new TabletServerStatus();
-      Map<String,TableInfo> tableMap = new HashMap<>();
-      tableMap.put(FOO.getId().canonical(), new TableInfo());
-      tableMap.put(BAR.getId().canonical(), new TableInfo());
-      tableMap.put(BAZ.getId().canonical(), new TableInfo());
-      status.setTableMap(tableMap);
-      current.put(new TServerInstance(base + i + ":9997", 1), status);
-    }
-    // now put all of the tablets on one server
-    for (Map.Entry<String,TServerInstance> entry : initialTableLocation.entrySet()) {
-      TabletServerStatus status = current.get(entry.getValue());
-      if (status != null) {
-        String tableId = getTableOperations().tableIdMap().get(entry.getKey());
-        status.getTableMap().get(tableId).setOnlineTablets(5);
-      }
-    }
-    return current;
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
deleted file mode 100644
index 02835e2..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java
+++ /dev/null
@@ -1,303 +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.server.master.balancer;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.1.0")
-public class DefaultLoadBalancerTest {
-
-  class FakeTServer {
-    List<KeyExtent> extents = new ArrayList<>();
-
-    TabletServerStatus getStatus() {
-      TabletServerStatus result = new TabletServerStatus();
-      result.tableMap = new HashMap<>();
-      for (KeyExtent extent : extents) {
-        TableId tableId = extent.tableId();
-        TableInfo info = result.tableMap.get(tableId.canonical());
-        if (info == null) {
-          result.tableMap.put(tableId.canonical(), info = new TableInfo());
-        }
-        info.onlineTablets++;
-        info.recs = info.onlineTablets;
-        info.ingestRate = 123.;
-        info.queryRate = 456.;
-      }
-      return result;
-    }
-  }
-
-  Map<TServerInstance,FakeTServer> servers = new HashMap<>();
-  Map<KeyExtent,TServerInstance> last = new HashMap<>();
-
-  class TestDefaultLoadBalancer extends DefaultLoadBalancer {
-
-    @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId table) {
-      List<TabletStats> result = new ArrayList<>();
-      for (KeyExtent extent : servers.get(tserver).extents) {
-        if (extent.tableId().equals(table)) {
-          result.add(new TabletStats(extent.toThrift(), null, null, null, 0L, 0., 0., 0));
-        }
-      }
-      return result;
-    }
-  }
-
-  @BeforeEach
-  public void setUp() {
-    last.clear();
-    servers.clear();
-  }
-
-  @Test
-  public void testAssignMigrations() {
-    servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), "a"),
-        new FakeTServer());
-    servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.2", 1234), "b"),
-        new FakeTServer());
-    servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.3", 1234), "c"),
-        new FakeTServer());
-    List<KeyExtent> metadataTable = new ArrayList<>();
-    String table = "t1";
-    metadataTable.add(makeExtent(table, null, null));
-    table = "t2";
-    metadataTable.add(makeExtent(table, "a", null));
-    metadataTable.add(makeExtent(table, null, "a"));
-    table = "t3";
-    metadataTable.add(makeExtent(table, "a", null));
-    metadataTable.add(makeExtent(table, "b", "a"));
-    metadataTable.add(makeExtent(table, "c", "b"));
-    metadataTable.add(makeExtent(table, "d", "c"));
-    metadataTable.add(makeExtent(table, "e", "d"));
-    metadataTable.add(makeExtent(table, null, "e"));
-    Collections.sort(metadataTable);
-
-    TestDefaultLoadBalancer balancer = new TestDefaultLoadBalancer();
-
-    SortedMap<TServerInstance,TabletServerStatus> current = new TreeMap<>();
-    for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) {
-      current.put(entry.getKey(), entry.getValue().getStatus());
-    }
-    assignTablets(metadataTable, servers, current, balancer);
-
-    // Verify that the counts on the tables are correct
-    Map<String,Integer> expectedCounts = new HashMap<>();
-    expectedCounts.put("t1", 1);
-    expectedCounts.put("t2", 1);
-    expectedCounts.put("t3", 2);
-    checkBalance(metadataTable, servers, expectedCounts);
-
-    // Rebalance once
-    for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) {
-      current.put(entry.getKey(), entry.getValue().getStatus());
-    }
-
-    // Nothing should happen, we are balanced
-    ArrayList<TabletMigration> out = new ArrayList<>();
-    balancer.getMigrations(current, out);
-    assertEquals(out.size(), 0);
-
-    // Take down a tabletServer
-    TServerInstance first = current.keySet().iterator().next();
-    current.remove(first);
-    FakeTServer remove = servers.remove(first);
-
-    // reassign offline extents
-    assignTablets(remove.extents, servers, current, balancer);
-    checkBalance(metadataTable, servers, null);
-  }
-
-  private void assignTablets(List<KeyExtent> metadataTable,
-      Map<TServerInstance,FakeTServer> servers,
-      SortedMap<TServerInstance,TabletServerStatus> status, TestDefaultLoadBalancer balancer) {
-    // Assign tablets
-    for (KeyExtent extent : metadataTable) {
-      TServerInstance assignment = balancer.getAssignment(status, last.get(extent));
-      assertNotNull(assignment);
-      assertFalse(servers.get(assignment).extents.contains(extent));
-      servers.get(assignment).extents.add(extent);
-      last.put(extent, assignment);
-    }
-  }
-
-  SortedMap<TServerInstance,TabletServerStatus>
-      getAssignments(Map<TServerInstance,FakeTServer> servers) {
-    SortedMap<TServerInstance,TabletServerStatus> result = new TreeMap<>();
-    for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) {
-      result.put(entry.getKey(), entry.getValue().getStatus());
-    }
-    return result;
-  }
-
-  @Test
-  public void testUnevenAssignment() {
-    for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) {
-      String cString = Character.toString(c);
-      HostAndPort fakeAddress = HostAndPort.fromParts("127.0.0.1", c);
-      TServerInstance tsi = new TServerInstance(fakeAddress, cString);
-      FakeTServer fakeTServer = new FakeTServer();
-      servers.put(tsi, fakeTServer);
-      fakeTServer.extents.add(makeExtent(cString, null, null));
-    }
-    // Put more tablets on one server, but not more than the number of servers
-    Entry<TServerInstance,FakeTServer> first = servers.entrySet().iterator().next();
-    first.getValue().extents.add(makeExtent("newTable", "a", null));
-    first.getValue().extents.add(makeExtent("newTable", "b", "a"));
-    first.getValue().extents.add(makeExtent("newTable", "c", "b"));
-    first.getValue().extents.add(makeExtent("newTable", "d", "c"));
-    first.getValue().extents.add(makeExtent("newTable", "e", "d"));
-    first.getValue().extents.add(makeExtent("newTable", "f", "e"));
-    first.getValue().extents.add(makeExtent("newTable", "g", "f"));
-    first.getValue().extents.add(makeExtent("newTable", "h", "g"));
-    first.getValue().extents.add(makeExtent("newTable", "i", null));
-    TestDefaultLoadBalancer balancer = new TestDefaultLoadBalancer();
-    Set<KeyExtent> migrations = Collections.emptySet();
-    int moved = 0;
-    // balance until we can't balance no more!
-    while (true) {
-      List<TabletMigration> migrationsOut = new ArrayList<>();
-      balancer.balance(getAssignments(servers), migrations, migrationsOut);
-      if (migrationsOut.isEmpty()) {
-        break;
-      }
-      for (TabletMigration migration : migrationsOut) {
-        if (servers.get(migration.oldServer).extents.remove(migration.tablet)) {
-          moved++;
-        }
-        servers.get(migration.newServer).extents.add(migration.tablet);
-      }
-    }
-    assertEquals(8, moved);
-  }
-
-  @Test
-  public void testUnevenAssignment2() {
-    // make 26 servers
-    for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) {
-      String cString = Character.toString(c);
-      HostAndPort fakeAddress = HostAndPort.fromParts("127.0.0.1", c);
-      TServerInstance tsi = new TServerInstance(fakeAddress, cString);
-      FakeTServer fakeTServer = new FakeTServer();
-      servers.put(tsi, fakeTServer);
-    }
-    // put 60 tablets on 25 of them
-    List<Entry<TServerInstance,FakeTServer>> shortList = new ArrayList<>(servers.entrySet());
-    Entry<TServerInstance,FakeTServer> shortServer = shortList.remove(0);
-    int c = 0;
-    for (int i = 0; i < 60; i++) {
-      for (Entry<TServerInstance,FakeTServer> entry : shortList) {
-        entry.getValue().extents.add(makeExtent("t" + c, null, null));
-      }
-    }
-    // put 10 on the that short server:
-    for (int i = 0; i < 10; i++) {
-      shortServer.getValue().extents.add(makeExtent("s" + i, null, null));
-    }
-
-    TestDefaultLoadBalancer balancer = new TestDefaultLoadBalancer();
-    Set<KeyExtent> migrations = Collections.emptySet();
-    int moved = 0;
-    // balance until we can't balance no more!
-    while (true) {
-      List<TabletMigration> migrationsOut = new ArrayList<>();
-      balancer.balance(getAssignments(servers), migrations, migrationsOut);
-      if (migrationsOut.isEmpty()) {
-        break;
-      }
-      for (TabletMigration migration : migrationsOut) {
-        if (servers.get(migration.oldServer).extents.remove(migration.tablet)) {
-          moved++;
-        }
-        last.remove(migration.tablet);
-        servers.get(migration.newServer).extents.add(migration.tablet);
-        last.put(migration.tablet, migration.newServer);
-      }
-    }
-    // average is 58, with 2 at 59: we need 48 more moved to the short server
-    assertEquals(48, moved);
-  }
-
-  private void checkBalance(List<KeyExtent> metadataTable, Map<TServerInstance,FakeTServer> servers,
-      Map<String,Integer> expectedCounts) {
-    // Verify they are spread evenly over the cluster
-    int average = metadataTable.size() / servers.size();
-    for (FakeTServer server : servers.values()) {
-      int diff = server.extents.size() - average;
-      if (diff < 0) {
-        fail("average number of tablets is " + average + " but a server has "
-            + server.extents.size());
-      }
-      if (diff > 1) {
-        fail("average number of tablets is " + average + " but a server has "
-            + server.extents.size());
-      }
-    }
-
-    if (expectedCounts != null) {
-      for (FakeTServer server : servers.values()) {
-        Map<String,Integer> counts = new HashMap<>();
-        server.extents.forEach(extent -> {
-          String t = extent.tableId().canonical();
-          counts.putIfAbsent(t, 0);
-          counts.put(t, counts.get(t) + 1);
-        });
-        counts.forEach((k, v) -> assertEquals(expectedCounts.get(k), v));
-      }
-    }
-  }
-
-  private static KeyExtent makeExtent(String table, String end, String prev) {
-    return new KeyExtent(TableId.of(table), toText(end), toText(prev));
-  }
-
-  private static Text toText(String value) {
-    if (value != null) {
-      return new Text(value);
-    }
-    return null;
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
deleted file mode 100644
index a25bfcf..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/GroupBalancerTest.java
+++ /dev/null
@@ -1,359 +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.server.master.balancer;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.function.Function;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.util.MapCounter;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.1.0")
-public class GroupBalancerTest {
-
-  private static final SecureRandom random = new SecureRandom();
-
-  private static Function<KeyExtent,String> partitioner = new Function<>() {
-
-    @Override
-    public String apply(KeyExtent input) {
-      return (input == null || input.endRow() == null) ? null
-          : input.endRow().toString().substring(0, 2);
-    }
-  };
-
-  public static class TabletServers {
-    private final Set<TServerInstance> tservers = new HashSet<>();
-    private final Map<KeyExtent,TServerInstance> tabletLocs = new HashMap<>();
-
-    public void addTservers(String... locs) {
-      for (String loc : locs) {
-        addTserver(loc);
-      }
-    }
-
-    public void addTserver(String loc) {
-      tservers.add(new TServerInstance(loc, 6));
-    }
-
-    public void addTablet(String er, String location) {
-      TServerInstance tsi = new TServerInstance(location, 6);
-      tabletLocs.put(new KeyExtent(TableId.of("b"), er == null ? null : new Text(er), null),
-          new TServerInstance(location, 6));
-      tservers.add(tsi);
-    }
-
-    public void balance() {
-      balance(10000);
-    }
-
-    public void balance(final int maxMigrations) {
-      GroupBalancer balancer = new GroupBalancer(TableId.of("1")) {
-
-        @Override
-        protected Map<KeyExtent,TServerInstance> getLocationProvider() {
-          return tabletLocs;
-        }
-
-        @Override
-        protected Function<KeyExtent,String> getPartitioner() {
-          return partitioner;
-        }
-
-        @Override
-        protected long getWaitTime() {
-          return 0;
-        }
-
-        @Override
-        protected int getMaxMigrations() {
-          return maxMigrations;
-        }
-      };
-
-      balance(balancer, maxMigrations);
-    }
-
-    public void balance(TabletBalancer balancer, int maxMigrations) {
-
-      while (true) {
-        Set<KeyExtent> migrations = new HashSet<>();
-        List<TabletMigration> migrationsOut = new ArrayList<>();
-        SortedMap<TServerInstance,TabletServerStatus> current = new TreeMap<>();
-
-        for (TServerInstance tsi : tservers) {
-          current.put(tsi, new TabletServerStatus());
-        }
-
-        balancer.balance(current, migrations, migrationsOut);
-
-        assertTrue(migrationsOut.size() <= (maxMigrations + 5),
-            "Max Migration exceeded " + maxMigrations + " " + migrationsOut.size());
-
-        for (TabletMigration tabletMigration : migrationsOut) {
-          assertEquals(tabletLocs.get(tabletMigration.tablet), tabletMigration.oldServer);
-          assertTrue(tservers.contains(tabletMigration.newServer));
-
-          tabletLocs.put(tabletMigration.tablet, tabletMigration.newServer);
-        }
-
-        if (migrationsOut.isEmpty()) {
-          break;
-        }
-      }
-
-      checkBalance();
-    }
-
-    void checkBalance() {
-      MapCounter<String> groupCounts = new MapCounter<>();
-      Map<TServerInstance,MapCounter<String>> tserverGroupCounts = new HashMap<>();
-
-      for (Entry<KeyExtent,TServerInstance> entry : tabletLocs.entrySet()) {
-        String group = partitioner.apply(entry.getKey());
-        TServerInstance loc = entry.getValue();
-
-        groupCounts.increment(group, 1);
-        MapCounter<String> tgc = tserverGroupCounts.get(loc);
-        if (tgc == null) {
-          tgc = new MapCounter<>();
-          tserverGroupCounts.put(loc, tgc);
-        }
-
-        tgc.increment(group, 1);
-      }
-
-      Map<String,Integer> expectedCounts = new HashMap<>();
-
-      int totalExtra = 0;
-      for (String group : groupCounts.keySet()) {
-        long groupCount = groupCounts.get(group);
-        totalExtra += groupCount % tservers.size();
-        expectedCounts.put(group, (int) (groupCount / tservers.size()));
-      }
-
-      // The number of extra tablets from all groups that each tserver must have.
-      int expectedExtra = totalExtra / tservers.size();
-      int maxExtraGroups = expectedExtra + ((totalExtra % tservers.size() > 0) ? 1 : 0);
-
-      for (Entry<TServerInstance,MapCounter<String>> entry : tserverGroupCounts.entrySet()) {
-        MapCounter<String> tgc = entry.getValue();
-        int tserverExtra = 0;
-        for (String group : groupCounts.keySet()) {
-          assertTrue(tgc.get(group) >= expectedCounts.get(group));
-          assertTrue(tgc.get(group) <= expectedCounts.get(group) + 1,
-              "Group counts not as expected group:" + group + " actual:" + tgc.get(group)
-                  + " expected:" + (expectedCounts.get(group) + 1) + " tserver:" + entry.getKey());
-          tserverExtra += tgc.get(group) - expectedCounts.get(group);
-        }
-
-        assertTrue(tserverExtra >= expectedExtra);
-        assertTrue(tserverExtra <= maxExtraGroups);
-      }
-    }
-  }
-
-  @Test
-  public void testSingleGroup() {
-
-    String[][] tests = {new String[] {"a", "b", "c", "d"}, new String[] {"a", "b", "c"},
-        new String[] {"a", "b", "c", "d", "e"}, new String[] {"a", "b", "c", "d", "e", "f", "g"},
-        new String[] {"a", "b", "c", "d", "e", "f", "g", "h"},
-        new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i"}, new String[] {"a"}};
-
-    for (String[] suffixes : tests) {
-      for (int maxTS = 1; maxTS <= 4; maxTS++) {
-        TabletServers tservers = new TabletServers();
-        tservers = new TabletServers();
-        int ts = 0;
-        for (String s : suffixes) {
-          tservers.addTablet("01" + s, "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-        }
-
-        tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
-        tservers.balance();
-        tservers.balance();
-      }
-    }
-  }
-
-  @Test
-  public void testTwoGroups() {
-    String[][] tests = {new String[] {"a", "b", "c", "d"}, new String[] {"a", "b", "c"},
-        new String[] {"a", "b", "c", "d", "e"}, new String[] {"a", "b", "c", "d", "e", "f", "g"},
-        new String[] {"a", "b", "c", "d", "e", "f", "g", "h"},
-        new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i"}, new String[] {"a"}};
-
-    for (String[] suffixes1 : tests) {
-      for (String[] suffixes2 : tests) {
-        for (int maxTS = 1; maxTS <= 4; maxTS++) {
-          TabletServers tservers = new TabletServers();
-          tservers = new TabletServers();
-          int ts = 0;
-          for (String s : suffixes1) {
-            tservers.addTablet("01" + s, "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-          }
-
-          for (String s : suffixes2) {
-            tservers.addTablet("02" + s, "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-          }
-
-          tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
-          tservers.balance();
-          tservers.balance();
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testThreeGroups() {
-    String[][] tests = {new String[] {"a", "b", "c", "d"}, new String[] {"a", "b", "c"},
-        new String[] {"a", "b", "c", "d", "e"}, new String[] {"a", "b", "c", "d", "e", "f", "g"},
-        new String[] {"a", "b", "c", "d", "e", "f", "g", "h"},
-        new String[] {"a", "b", "c", "d", "e", "f", "g", "h", "i"}, new String[] {"a"}};
-
-    for (String[] suffixes1 : tests) {
-      for (String[] suffixes2 : tests) {
-        for (String[] suffixes3 : tests) {
-          for (int maxTS = 1; maxTS <= 4; maxTS++) {
-            TabletServers tservers = new TabletServers();
-            tservers = new TabletServers();
-            int ts = 0;
-            for (String s : suffixes1) {
-              tservers.addTablet("01" + s, "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-            }
-
-            for (String s : suffixes2) {
-              tservers.addTablet("02" + s, "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-            }
-
-            for (String s : suffixes3) {
-              tservers.addTablet("03" + s, "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-            }
-
-            tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
-            tservers.balance();
-            tservers.balance();
-          }
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testManySingleTabletGroups() {
-
-    for (int numGroups = 1; numGroups <= 13; numGroups++) {
-      for (int maxTS = 1; maxTS <= 4; maxTS++) {
-        TabletServers tservers = new TabletServers();
-        tservers = new TabletServers();
-        int ts = 0;
-
-        for (int group = 1; group <= numGroups; group++) {
-          tservers.addTablet(String.format("%02d:p", group),
-              "192.168.1." + ((ts++ % maxTS) + 1) + ":9997");
-        }
-
-        tservers.addTservers("192.168.1.2:9997", "192.168.1.3:9997", "192.168.1.4:9997");
-
-        tservers.balance();
-        tservers.balance();
-      }
-    }
-  }
-
-  @Test
-  public void testMaxMigrations() {
-
-    for (int max : new int[] {1, 2, 3, 7, 10, 30}) {
-      TabletServers tservers = new TabletServers();
-
-      for (int i = 1; i <= 9; i++) {
-        tservers.addTablet("01" + i, "192.168.1.1:9997");
-      }
-
-      for (int i = 1; i <= 4; i++) {
-        tservers.addTablet("02" + i, "192.168.1.2:9997");
-      }
-
-      for (int i = 1; i <= 5; i++) {
-        tservers.addTablet("03" + i, "192.168.1.3:9997");
-      }
-
-      tservers.addTservers("192.168.1.4:9997", "192.168.1.5:9997");
-
-      tservers.balance(max);
-    }
-  }
-
-  @Test
-  public void bigTest() {
-    TabletServers tservers = new TabletServers();
-
-    for (int g = 1; g <= 60; g++) {
-      for (int t = 1; t <= 241; t++) {
-        tservers.addTablet(String.format("%02d:%d", g, t),
-            "192.168.1." + (random.nextInt(249) + 1) + ":9997");
-      }
-    }
-
-    for (int i = 1; i <= 250; i++) {
-      tservers.addTserver("192.168.1." + i + ":9997");
-    }
-
-    tservers.balance(1000);
-  }
-
-  @Test
-  public void bigTest2() {
-    TabletServers tservers = new TabletServers();
-
-    for (int g = 1; g <= 60; g++) {
-      for (int t = 1; t <= random.nextInt(1000); t++) {
-        tservers.addTablet(String.format("%02d:%d", g, t),
-            "192.168.1." + (random.nextInt(249) + 1) + ":9997");
-      }
-    }
-
-    for (int i = 1; i <= 250; i++) {
-      tservers.addTserver("192.168.1." + i + ":9997");
-    }
-
-    tservers.balance(1000);
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
deleted file mode 100644
index 186ee5f..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
+++ /dev/null
@@ -1,132 +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.server.master.balancer;
-
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.1.0")
-public class HostRegexTableLoadBalancerReconfigurationTest
-    extends BaseHostRegexTableLoadBalancerTest {
-
-  private Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-
-  @Test
-  public void testConfigurationChanges() {
-    ServerContext context1 = createMockContext();
-    replay(context1);
-    final TestServerConfigurationFactory factory = new TestServerConfigurationFactory(context1);
-    ServerContext context2 = createMockContext();
-    expect(context2.getConfiguration()).andReturn(factory.getSystemConfiguration()).anyTimes();
-    expect(context2.getTableConfiguration(FOO.getId()))
-        .andReturn(factory.getTableConfiguration(FOO.getId())).anyTimes();
-    expect(context2.getTableConfiguration(BAR.getId()))
-        .andReturn(factory.getTableConfiguration(BAR.getId())).anyTimes();
-    expect(context2.getTableConfiguration(BAZ.getId()))
-        .andReturn(factory.getTableConfiguration(BAZ.getId())).anyTimes();
-    replay(context2);
-    init(context2);
-    Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
-    for (List<KeyExtent> extents : tableExtents.values()) {
-      for (KeyExtent ke : extents) {
-        unassigned.put(ke, null);
-      }
-    }
-    this.getAssignments(Collections.unmodifiableSortedMap(allTabletServers),
-        Collections.unmodifiableMap(unassigned), assignments);
-    assertEquals(15, assignments.size());
-    // Ensure unique tservers
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      for (Entry<KeyExtent,TServerInstance> e2 : assignments.entrySet()) {
-        if (e.getKey().equals(e2.getKey())) {
-          continue;
-        }
-        if (e.getValue().equals(e2.getValue())) {
-          fail("Assignment failure. " + e.getKey() + " and " + e2.getKey()
-              + " are assigned to the same host: " + e.getValue());
-        }
-      }
-    }
-    // Ensure assignments are correct
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      if (!tabletInBounds(e.getKey(), e.getValue())) {
-        fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
-      }
-    }
-    Set<KeyExtent> migrations = new HashSet<>();
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    // Wait to trigger the out of bounds check which will call our version of
-    // getOnlineTabletsForTable
-    UtilWaitThread.sleep(3000);
-    this.balance(Collections.unmodifiableSortedMap(allTabletServers), migrations, migrationsOut);
-    assertEquals(0, migrationsOut.size());
-    // Change property, simulate call by TableConfWatcher
-
-    ((ConfigurationCopy) factory.getSystemConfiguration())
-        .set(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r01.*");
-
-    // Wait to trigger the out of bounds check and the repool check
-    UtilWaitThread.sleep(10000);
-    this.balance(Collections.unmodifiableSortedMap(allTabletServers), migrations, migrationsOut);
-    assertEquals(5, migrationsOut.size());
-    for (TabletMigration migration : migrationsOut) {
-      assertTrue(migration.newServer.getHost().startsWith("192.168.0.1")
-          || migration.newServer.getHost().startsWith("192.168.0.2")
-          || migration.newServer.getHost().startsWith("192.168.0.3")
-          || migration.newServer.getHost().startsWith("192.168.0.4")
-          || migration.newServer.getHost().startsWith("192.168.0.5"));
-    }
-  }
-
-  @Override
-  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
-    List<TabletStats> tablets = new ArrayList<>();
-    // Report assignment information
-    for (Entry<KeyExtent,TServerInstance> e : this.assignments.entrySet()) {
-      if (e.getValue().equals(tserver) && e.getKey().tableId().equals(tableId)) {
-        TabletStats ts = new TabletStats();
-        ts.setExtent(e.getKey().toThrift());
-        tablets.add(ts);
-      }
-    }
-    return tablets;
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
deleted file mode 100644
index 8a40716..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
+++ /dev/null
@@ -1,480 +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.server.master.balancer;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.regex.Pattern;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.1.0")
-public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalancerTest {
-
-  public void init() {
-    ServerContext context1 = createMockContext();
-    replay(context1);
-    final TestServerConfigurationFactory factory = new TestServerConfigurationFactory(context1);
-    initFactory(factory);
-  }
-
-  private void initFactory(ServerConfigurationFactory factory) {
-    ServerContext context = createMockContext();
-    expect(context.getConfiguration()).andReturn(factory.getSystemConfiguration()).anyTimes();
-    expect(context.getTableConfiguration(FOO.getId()))
-        .andReturn(factory.getTableConfiguration(FOO.getId())).anyTimes();
-    expect(context.getTableConfiguration(BAR.getId()))
-        .andReturn(factory.getTableConfiguration(BAR.getId())).anyTimes();
-    expect(context.getTableConfiguration(BAZ.getId()))
-        .andReturn(factory.getTableConfiguration(BAZ.getId())).anyTimes();
-    replay(context);
-    init(context);
-  }
-
-  @Test
-  public void testInit() {
-    init();
-    assertEquals(7000, this.getOobCheckMillis(), "OOB check interval value is incorrect");
-    assertEquals(4, this.getMaxMigrations(), "Max migrations is incorrect");
-    assertEquals(10, this.getMaxOutstandingMigrations(), "Max outstanding migrations is incorrect");
-    assertFalse(isIpBasedRegex());
-    Map<String,Pattern> patterns = this.getPoolNameToRegexPattern();
-    assertEquals(2, patterns.size());
-    assertTrue(patterns.containsKey(FOO.getTableName()));
-    assertEquals(Pattern.compile("r01.*").pattern(), patterns.get(FOO.getTableName()).pattern());
-    assertTrue(patterns.containsKey(BAR.getTableName()));
-    assertEquals(Pattern.compile("r02.*").pattern(), patterns.get(BAR.getTableName()).pattern());
-  }
-
-  @Test
-  public void testBalance() {
-    init();
-    Set<KeyExtent> migrations = new HashSet<>();
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    long wait = this.balance(Collections.unmodifiableSortedMap(createCurrent(15)), migrations,
-        migrationsOut);
-    assertEquals(20000, wait);
-    // should balance four tablets in one of the tables before reaching max
-    assertEquals(4, migrationsOut.size());
-
-    // now balance again passing in the new migrations
-    for (TabletMigration m : migrationsOut) {
-      migrations.add(m.tablet);
-    }
-    migrationsOut.clear();
-    wait = this.balance(Collections.unmodifiableSortedMap(createCurrent(15)), migrations,
-        migrationsOut);
-    assertEquals(20000, wait);
-    // should balance four tablets in one of the other tables before reaching max
-    assertEquals(4, migrationsOut.size());
-
-    // now balance again passing in the new migrations
-    for (TabletMigration m : migrationsOut) {
-      migrations.add(m.tablet);
-    }
-    migrationsOut.clear();
-    wait = this.balance(Collections.unmodifiableSortedMap(createCurrent(15)), migrations,
-        migrationsOut);
-    assertEquals(20000, wait);
-    // should balance four tablets in one of the other tables before reaching max
-    assertEquals(4, migrationsOut.size());
-
-    // now balance again passing in the new migrations
-    for (TabletMigration m : migrationsOut) {
-      migrations.add(m.tablet);
-    }
-    migrationsOut.clear();
-    wait = this.balance(Collections.unmodifiableSortedMap(createCurrent(15)), migrations,
-        migrationsOut);
-    assertEquals(20000, wait);
-    // no more balancing to do
-    assertEquals(0, migrationsOut.size());
-  }
-
-  @Test
-  public void testBalanceWithTooManyOutstandingMigrations() {
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    init();
-    // lets say we already have migrations ongoing for the FOO and BAR table extends (should be 5 of
-    // each of them) for a total of 10
-    Set<KeyExtent> migrations = new HashSet<>();
-    migrations.addAll(tableExtents.get(FOO.getTableName()));
-    migrations.addAll(tableExtents.get(BAR.getTableName()));
-    long wait = this.balance(Collections.unmodifiableSortedMap(createCurrent(15)), migrations,
-        migrationsOut);
-    assertEquals(20000, wait);
-    // no migrations should have occurred as 10 is the maxOutstandingMigrations
-    assertEquals(0, migrationsOut.size());
-  }
-
-  @Test
-  public void testSplitCurrentByRegexUsingHostname() {
-    init();
-    Map<String,SortedMap<TServerInstance,TabletServerStatus>> groups =
-        this.splitCurrentByRegex(createCurrent(15));
-    assertEquals(3, groups.size());
-    assertTrue(groups.containsKey(FOO.getTableName()));
-    SortedMap<TServerInstance,TabletServerStatus> fooHosts = groups.get(FOO.getTableName());
-    assertEquals(5, fooHosts.size());
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.1:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.2:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.3:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.4:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.5:9997", 1)));
-    assertTrue(groups.containsKey(BAR.getTableName()));
-    SortedMap<TServerInstance,TabletServerStatus> barHosts = groups.get(BAR.getTableName());
-    assertEquals(5, barHosts.size());
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.6:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.7:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.8:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.9:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.10:9997", 1)));
-    assertTrue(groups.containsKey(DEFAULT_POOL));
-    SortedMap<TServerInstance,TabletServerStatus> defHosts = groups.get(DEFAULT_POOL);
-    assertEquals(5, defHosts.size());
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.11:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.12:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.13:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.14:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.15:9997", 1)));
-  }
-
-  @Test
-  public void testSplitCurrentByRegexUsingOverlappingPools() {
-    ServerContext context = createMockContext();
-    replay(context);
-    initFactory(new TestServerConfigurationFactory(context) {
-      @Override
-      public synchronized AccumuloConfiguration getSystemConfiguration() {
-        HashMap<String,String> props = new HashMap<>(DEFAULT_TABLE_PROPERTIES);
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(), "r.*");
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(),
-            "r01.*|r02.*");
-        return new ConfigurationCopy(props);
-      }
-    });
-    Map<String,SortedMap<TServerInstance,TabletServerStatus>> groups =
-        this.splitCurrentByRegex(createCurrent(15));
-
-    // Groups foo, bar, and the default pool which contains all known hosts
-    assertEquals(3, groups.size());
-    assertTrue(groups.containsKey(FOO.getTableName()));
-    assertTrue(groups.containsKey(DEFAULT_POOL));
-    for (String pool : new String[] {FOO.getTableName(), DEFAULT_POOL}) {
-      SortedMap<TServerInstance,TabletServerStatus> fooHosts = groups.get(pool);
-      assertEquals(15, fooHosts.size());
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.1:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.2:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.3:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.4:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.5:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.6:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.7:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.8:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.9:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.10:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.11:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.12:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.13:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.14:9997", 1)));
-      assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.15:9997", 1)));
-    }
-
-    assertTrue(groups.containsKey(BAR.getTableName()));
-    SortedMap<TServerInstance,TabletServerStatus> barHosts = groups.get(BAR.getTableName());
-    assertEquals(10, barHosts.size());
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.1:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.2:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.3:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.4:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.5:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.6:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.7:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.8:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.9:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.10:9997", 1)));
-  }
-
-  @Test
-  public void testSplitCurrentByRegexUsingIP() {
-    ServerContext context = createMockContext();
-    replay(context);
-    initFactory(new TestServerConfigurationFactory(context) {
-      @Override
-      public synchronized AccumuloConfiguration getSystemConfiguration() {
-        HashMap<String,String> props = new HashMap<>();
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "30s");
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_REGEX_USING_IPS_KEY, "true");
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(),
-            "192\\.168\\.0\\.[1-5]");
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(),
-            "192\\.168\\.0\\.[6-9]|192\\.168\\.0\\.10");
-        return new ConfigurationCopy(props);
-      }
-    });
-    assertTrue(isIpBasedRegex());
-    Map<String,SortedMap<TServerInstance,TabletServerStatus>> groups =
-        this.splitCurrentByRegex(createCurrent(15));
-    assertEquals(3, groups.size());
-    assertTrue(groups.containsKey(FOO.getTableName()));
-    SortedMap<TServerInstance,TabletServerStatus> fooHosts = groups.get(FOO.getTableName());
-    assertEquals(5, fooHosts.size());
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.1:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.2:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.3:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.4:9997", 1)));
-    assertTrue(fooHosts.containsKey(new TServerInstance("192.168.0.5:9997", 1)));
-    assertTrue(groups.containsKey(BAR.getTableName()));
-    SortedMap<TServerInstance,TabletServerStatus> barHosts = groups.get(BAR.getTableName());
-    assertEquals(5, barHosts.size());
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.6:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.7:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.8:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.9:9997", 1)));
-    assertTrue(barHosts.containsKey(new TServerInstance("192.168.0.10:9997", 1)));
-    assertTrue(groups.containsKey(DEFAULT_POOL));
-    SortedMap<TServerInstance,TabletServerStatus> defHosts = groups.get(DEFAULT_POOL);
-    assertEquals(5, defHosts.size());
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.11:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.12:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.13:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.14:9997", 1)));
-    assertTrue(defHosts.containsKey(new TServerInstance("192.168.0.15:9997", 1)));
-  }
-
-  @Test
-  public void testAllUnassigned() {
-    init();
-    Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-    Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
-    for (List<KeyExtent> extents : tableExtents.values()) {
-      for (KeyExtent ke : extents) {
-        unassigned.put(ke, null);
-      }
-    }
-    this.getAssignments(Collections.unmodifiableSortedMap(allTabletServers),
-        Collections.unmodifiableMap(unassigned), assignments);
-    assertEquals(15, assignments.size());
-    // Ensure unique tservers
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      for (Entry<KeyExtent,TServerInstance> e2 : assignments.entrySet()) {
-        if (e.getKey().equals(e2.getKey())) {
-          continue;
-        }
-        if (e.getValue().equals(e2.getValue())) {
-          fail("Assignment failure");
-        }
-      }
-    }
-    // Ensure assignments are correct
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      if (!tabletInBounds(e.getKey(), e.getValue())) {
-        fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
-      }
-    }
-  }
-
-  @Test
-  public void testAllAssigned() {
-    init();
-    Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-    Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
-    this.getAssignments(Collections.unmodifiableSortedMap(allTabletServers),
-        Collections.unmodifiableMap(unassigned), assignments);
-    assertEquals(0, assignments.size());
-  }
-
-  @Test
-  public void testPartiallyAssigned() {
-    init();
-    Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-    Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
-    int i = 0;
-    for (List<KeyExtent> extents : tableExtents.values()) {
-      for (KeyExtent ke : extents) {
-        if ((i % 2) == 0) {
-          unassigned.put(ke, null);
-        }
-        i++;
-      }
-    }
-    this.getAssignments(Collections.unmodifiableSortedMap(allTabletServers),
-        Collections.unmodifiableMap(unassigned), assignments);
-    assertEquals(unassigned.size(), assignments.size());
-    // Ensure unique tservers
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      for (Entry<KeyExtent,TServerInstance> e2 : assignments.entrySet()) {
-        if (e.getKey().equals(e2.getKey())) {
-          continue;
-        }
-        if (e.getValue().equals(e2.getValue())) {
-          fail("Assignment failure");
-        }
-      }
-    }
-    // Ensure assignments are correct
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      if (!tabletInBounds(e.getKey(), e.getValue())) {
-        fail("tablet not in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
-      }
-    }
-  }
-
-  @Test
-  public void testUnassignedWithNoTServers() {
-    init();
-    Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-    Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
-    for (KeyExtent ke : tableExtents.get(BAR.getTableName())) {
-      unassigned.put(ke, null);
-    }
-    SortedMap<TServerInstance,TabletServerStatus> current = createCurrent(15);
-    // Remove the BAR tablet servers from current
-    List<TServerInstance> removals = new ArrayList<>();
-    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-      if (e.getKey().getHost().equals("192.168.0.6") || e.getKey().getHost().equals("192.168.0.7")
-          || e.getKey().getHost().equals("192.168.0.8")
-          || e.getKey().getHost().equals("192.168.0.9")
-          || e.getKey().getHost().equals("192.168.0.10")) {
-        removals.add(e.getKey());
-      }
-    }
-    for (TServerInstance r : removals) {
-      current.remove(r);
-    }
-    this.getAssignments(Collections.unmodifiableSortedMap(current),
-        Collections.unmodifiableMap(unassigned), assignments);
-    assertEquals(unassigned.size(), assignments.size());
-    // Ensure assignments are correct
-    // Ensure tablets are assigned in default pool
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      if (tabletInBounds(e.getKey(), e.getValue())) {
-        fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
-      }
-    }
-  }
-
-  @Test
-  public void testUnassignedWithNoDefaultPool() {
-    init();
-    Map<KeyExtent,TServerInstance> assignments = new HashMap<>();
-    Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
-    for (KeyExtent ke : tableExtents.get(BAR.getTableName())) {
-      unassigned.put(ke, null);
-    }
-
-    SortedMap<TServerInstance,TabletServerStatus> current = createCurrent(15);
-    // Remove the BAR tablet servers and default pool from current
-    List<TServerInstance> removals = new ArrayList<>();
-    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-      if (e.getKey().getHost().equals("192.168.0.6") || e.getKey().getHost().equals("192.168.0.7")
-          || e.getKey().getHost().equals("192.168.0.8")
-          || e.getKey().getHost().equals("192.168.0.9")
-          || e.getKey().getHost().equals("192.168.0.10")
-          || e.getKey().getHost().equals("192.168.0.11")
-          || e.getKey().getHost().equals("192.168.0.12")
-          || e.getKey().getHost().equals("192.168.0.13")
-          || e.getKey().getHost().equals("192.168.0.14")
-          || e.getKey().getHost().equals("192.168.0.15")) {
-        removals.add(e.getKey());
-      }
-    }
-
-    for (TServerInstance r : removals) {
-      current.remove(r);
-    }
-
-    this.getAssignments(Collections.unmodifiableSortedMap(current),
-        Collections.unmodifiableMap(unassigned), assignments);
-    assertEquals(unassigned.size(), assignments.size());
-
-    // Ensure tablets are assigned in default pool
-    for (Entry<KeyExtent,TServerInstance> e : assignments.entrySet()) {
-      if (tabletInBounds(e.getKey(), e.getValue())) {
-        fail("tablet unexpectedly in bounds: " + e.getKey() + " -> " + e.getValue().getHost());
-      }
-    }
-  }
-
-  @Test
-  public void testOutOfBoundsTablets() {
-    init();
-    // Wait to trigger the out of bounds check which will call our version of
-    // getOnlineTabletsForTable
-    UtilWaitThread.sleep(11000);
-    Set<KeyExtent> migrations = new HashSet<>();
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    this.balance(createCurrent(15), migrations, migrationsOut);
-    assertEquals(2, migrationsOut.size());
-  }
-
-  @Override
-  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
-    // Report incorrect information so that balance will create an assignment
-    List<TabletStats> tablets = new ArrayList<>();
-    if (tableId.equals(BAR.getId()) && tserver.getHost().equals("192.168.0.1")) {
-      // Report that we have a bar tablet on this server
-      TKeyExtent tke = new TKeyExtent();
-      tke.setTable(BAR.getId().canonical().getBytes(UTF_8));
-      tke.setEndRow("11".getBytes());
-      tke.setPrevEndRow("10".getBytes());
-      TabletStats ts = new TabletStats();
-      ts.setExtent(tke);
-      tablets.add(ts);
-    } else if (tableId.equals(FOO.getId()) && tserver.getHost().equals("192.168.0.6")) {
-      // Report that we have a foo tablet on this server
-      TKeyExtent tke = new TKeyExtent();
-      tke.setTable(FOO.getId().canonical().getBytes(UTF_8));
-      tke.setEndRow("1".getBytes());
-      tke.setPrevEndRow("0".getBytes());
-      TabletStats ts = new TabletStats();
-      ts.setExtent(tke);
-      tablets.add(ts);
-    }
-    return tablets;
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
deleted file mode 100644
index de9718b..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.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.server.master.balancer;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.server.MockServerContext;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.master.state.TabletMigration;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.Test;
-
-@Deprecated(since = "2.1.0")
-public class TableLoadBalancerTest {
-
-  private static Map<String,String> TABLE_ID_MAP = Map.of("t1", "a1", "t2", "b12", "t3", "c4");
-
-  private static TServerInstance mkts(String address, String session) {
-    return new TServerInstance(HostAndPort.fromParts(address, 1234), session);
-  }
-
-  private static TabletServerStatus status(Object... config) {
-    TabletServerStatus result = new TabletServerStatus();
-    result.tableMap = new HashMap<>();
-    String tablename = null;
-    for (Object c : config) {
-      if (c instanceof String) {
-        tablename = (String) c;
-      } else {
-        TableInfo info = new TableInfo();
-        int count = (Integer) c;
-        info.onlineTablets = count;
-        info.tablets = count;
-        result.tableMap.put(tablename, info);
-      }
-    }
-    return result;
-  }
-
-  static SortedMap<TServerInstance,TabletServerStatus> state;
-
-  static List<TabletStats> generateFakeTablets(TServerInstance tserver, TableId tableId) {
-    List<TabletStats> result = new ArrayList<>();
-    TabletServerStatus tableInfo = state.get(tserver);
-    // generate some fake tablets
-    for (int i = 0; i < tableInfo.tableMap.get(tableId.canonical()).onlineTablets; i++) {
-      TabletStats stats = new TabletStats();
-      stats.extent =
-          new KeyExtent(tableId, new Text(tserver.getHost() + String.format("%03d", i + 1)),
-              new Text(tserver.getHost() + String.format("%03d", i))).toThrift();
-      result.add(stats);
-    }
-    return result;
-  }
-
-  static class DefaultLoadBalancer
-      extends org.apache.accumulo.server.master.balancer.DefaultLoadBalancer {
-
-    public DefaultLoadBalancer(TableId table) {
-      super(table);
-    }
-
-    @Override
-    public void init(ServerContext context) {}
-
-    @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
-      return generateFakeTablets(tserver, tableId);
-    }
-  }
-
-  // ugh... so wish I had provided mock objects to the LoadBalancer in the manager
-  class TableLoadBalancer extends org.apache.accumulo.server.master.balancer.TableLoadBalancer {
-
-    // use our new classname to test class loading
-    @Override
-    protected String getLoadBalancerClassNameForTable(TableId table) {
-      return DefaultLoadBalancer.class.getName();
-    }
-
-    // we don't have real tablet servers to ask: invent some online tablets
-    @Override
-    public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, TableId tableId) {
-      return generateFakeTablets(tserver, tableId);
-    }
-
-    @Override
-    protected TableOperations getTableOperations() {
-      TableOperations tops = createMock(TableOperations.class);
-      expect(tops.tableIdMap()).andReturn(TABLE_ID_MAP).anyTimes();
-      replay(tops);
-      return tops;
-    }
-  }
-
-  private ServerContext createMockContext() {
-    final InstanceId instanceId =
-        InstanceId.of(UUID.nameUUIDFromBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}));
-    return MockServerContext.getWithZK(instanceId, "10.0.0.1:1234", 30_000);
-  }
-
-  @Test
-  public void test() {
-    final ServerContext context = createMockContext();
-    TableConfiguration conf = createMock(TableConfiguration.class);
-    // Eclipse might show @SuppressWarnings("removal") as unnecessary.
-    // Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-    @SuppressWarnings("removal")
-    Property TABLE_CLASSPATH = Property.TABLE_CLASSPATH;
-    expect(conf.resolve(Property.TABLE_CLASSLOADER_CONTEXT, TABLE_CLASSPATH))
-        .andReturn(Property.TABLE_CLASSLOADER_CONTEXT).anyTimes();
-    expect(conf.get(Property.TABLE_CLASSLOADER_CONTEXT)).andReturn("").anyTimes();
-    expect(context.getTableConfiguration(EasyMock.anyObject())).andReturn(conf).anyTimes();
-    replay(context, conf);
-
-    String t1Id = TABLE_ID_MAP.get("t1"), t2Id = TABLE_ID_MAP.get("t2"),
-        t3Id = TABLE_ID_MAP.get("t3");
-    state = new TreeMap<>();
-    TServerInstance svr = mkts("10.0.0.1", "0x01020304");
-    state.put(svr, status(t1Id, 10, t2Id, 10, t3Id, 10));
-
-    Set<KeyExtent> migrations = Collections.emptySet();
-    List<TabletMigration> migrationsOut = new ArrayList<>();
-    TableLoadBalancer tls = new TableLoadBalancer();
-    tls.init(context);
-    tls.balance(state, migrations, migrationsOut);
-    assertEquals(0, migrationsOut.size());
-
-    state.put(mkts("10.0.0.2", "0x02030405"), status());
-    tls = new TableLoadBalancer();
-    tls.init(context);
-    tls.balance(state, migrations, migrationsOut);
-    int count = 0;
-    Map<TableId,Integer> movedByTable = new HashMap<>();
-    movedByTable.put(TableId.of(t1Id), 0);
-    movedByTable.put(TableId.of(t2Id), 0);
-    movedByTable.put(TableId.of(t3Id), 0);
-    for (TabletMigration migration : migrationsOut) {
-      if (migration.oldServer.equals(svr)) {
-        count++;
-      }
-      TableId key = migration.tablet.tableId();
-      movedByTable.put(key, movedByTable.get(key) + 1);
-    }
-    assertEquals(15, count);
-    for (Integer moved : movedByTable.values()) {
-      assertEquals(5, moved.intValue());
-    }
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
index 04ec811..efbfbb0 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
@@ -40,7 +40,7 @@
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.util.Encoding;
 import org.apache.accumulo.server.MockServerContext;
 import org.apache.accumulo.server.ServerContext;
@@ -90,8 +90,8 @@
     ProblemReport r2 = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, null);
     assertEquals(r, r2);
     assertEquals(r2, r);
-    ProblemReport rx1 =
-        new ProblemReport(MetadataTable.ID, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    ProblemReport rx1 = new ProblemReport(AccumuloTable.METADATA.tableId(), ProblemType.FILE_READ,
+        RESOURCE, SERVER, null);
     assertNotEquals(r, rx1);
     ProblemReport rx2 = new ProblemReport(TABLE_ID, ProblemType.FILE_WRITE, RESOURCE, SERVER, null);
     assertNotEquals(r, rx2);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationUtilTest.java
deleted file mode 100644
index d39d45e..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationUtilTest.java
+++ /dev/null
@@ -1,115 +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.server.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.server.ServerContext;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class ReplicationUtilTest {
-
-  ServerContext context;
-  AccumuloConfiguration conf;
-  Map<String,String> confEntries;
-  ReplicaSystemFactory factory;
-  ReplicationUtil util;
-
-  @BeforeEach
-  public void setup() {
-    context = EasyMock.createMock(ServerContext.class);
-    conf = EasyMock.createMock(AccumuloConfiguration.class);
-    EasyMock.expect(context.getConfiguration()).andReturn(conf).anyTimes();
-    factory = new ReplicaSystemFactory();
-
-    util = new ReplicationUtil(context, factory);
-    confEntries = new HashMap<>();
-  }
-
-  @Test
-  public void testUserNamePassword() {
-    final String peerName = "peer";
-    final String systemImpl = "my.replica.system.impl";
-    final String config = "accumulo_peer,remote_host:2181";
-    final String peerDefinition = systemImpl + "," + config;
-    confEntries.put(Property.REPLICATION_PEER_USER.getKey() + peerName, "user");
-    confEntries.put(Property.REPLICATION_PEER_PASSWORD.getKey() + peerName, "password");
-    confEntries.put(Property.REPLICATION_PEERS.getKey() + peerName, peerDefinition);
-    ReplicaSystem system = EasyMock.createMock(ReplicaSystem.class);
-
-    // Return out our map of data
-    EasyMock.expect(conf.getAllPropertiesWithPrefix(Property.REPLICATION_PEERS))
-        .andReturn(confEntries);
-
-    // Switch to replay
-    EasyMock.replay(context, conf, system);
-
-    // Get the peers from our map
-    Map<String,String> peers = util.getPeers();
-
-    // Verify the mocked calls
-    EasyMock.verify(context, conf, system);
-
-    // Assert one peer with expected class name and configuration
-    assertEquals(1, peers.size());
-    Entry<String,String> peer = peers.entrySet().iterator().next();
-    assertEquals(peerName, peer.getKey());
-    assertEquals(systemImpl, peer.getValue());
-  }
-
-  @Test
-  public void testUserNameKeytab() {
-    final String peerName = "peer";
-    final String systemImpl = "my.replica.system.impl";
-    final String config = "accumulo_peer,remote_host:2181";
-    final String peerDefinition = systemImpl + "," + config;
-    confEntries.put(Property.REPLICATION_PEER_USER.getKey() + peerName, "user");
-    confEntries.put(Property.REPLICATION_PEER_KEYTAB.getKey() + peerName, "/path/to/keytab");
-    confEntries.put(Property.REPLICATION_PEERS.getKey() + peerName, peerDefinition);
-    ReplicaSystem system = EasyMock.createMock(ReplicaSystem.class);
-
-    // Return out our map of data
-    EasyMock.expect(conf.getAllPropertiesWithPrefix(Property.REPLICATION_PEERS))
-        .andReturn(confEntries);
-
-    // Switch to replay
-    EasyMock.replay(context, conf, system);
-
-    // Get the peers from our map
-    Map<String,String> peers = util.getPeers();
-
-    // Verify the mocked calls
-    EasyMock.verify(context, conf, system);
-
-    // Assert one peer with expected class name and configuration
-    assertEquals(1, peers.size());
-    Entry<String,String> peer = peers.entrySet().iterator().next();
-    assertEquals(peerName, peer.getKey());
-    assertEquals(systemImpl, peer.getValue());
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
deleted file mode 100644
index 974617d..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
+++ /dev/null
@@ -1,295 +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.server.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertSame;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.IteratorSetting.Column;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.iterators.Combiner;
-import org.apache.accumulo.core.iterators.DevNull;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class StatusCombinerTest {
-
-  private StatusCombiner combiner;
-  private Key key;
-  private Status.Builder builder;
-
-  private static class TestIE implements IteratorEnvironment {
-    @Override
-    public IteratorScope getIteratorScope() {
-      return IteratorScope.scan;
-    }
-  }
-
-  @BeforeEach
-  public void initCombiner() throws IOException {
-    key = new Key();
-    combiner = new StatusCombiner();
-    builder = Status.newBuilder();
-    IteratorSetting cfg = new IteratorSetting(50, StatusCombiner.class);
-    Combiner.setColumns(cfg, Collections.singletonList(new Column(StatusSection.NAME)));
-    combiner.init(new DevNull(), cfg.getOptions(), new TestIE());
-  }
-
-  @Test
-  public void returnsSameObject() {
-    Status status = StatusUtil.ingestedUntil(10);
-    // When combining only one message, we should get back the same instance
-    Status ret = combiner.typedReduce(key, Collections.singleton(status).iterator());
-    assertEquals(status, ret);
-    assertSame(status, ret);
-  }
-
-  @Test
-  public void newStatusWithNewIngest() {
-    Status orig = StatusUtil.fileCreated(100);
-    Status status = StatusUtil.replicatedAndIngested(10, 20);
-    Status ret = combiner.typedReduce(key, Arrays.asList(orig, status).iterator());
-    assertEquals(10L, ret.getBegin());
-    assertEquals(20L, ret.getEnd());
-    assertEquals(100L, ret.getCreatedTime());
-    assertFalse(ret.getClosed());
-  }
-
-  @Test
-  public void newStatusWithNewIngestSingleBuilder() {
-    Status orig = StatusUtil.fileCreated(100);
-    Status status = StatusUtil.replicatedAndIngested(builder, 10, 20);
-    Status ret = combiner.typedReduce(key, Arrays.asList(orig, status).iterator());
-    assertEquals(10L, ret.getBegin());
-    assertEquals(20L, ret.getEnd());
-    assertEquals(100L, ret.getCreatedTime());
-    assertFalse(ret.getClosed());
-  }
-
-  @Test
-  public void commutativeNewFile() {
-    Status newFile = StatusUtil.fileCreated(100), firstSync = StatusUtil.ingestedUntil(100),
-        secondSync = StatusUtil.ingestedUntil(200);
-
-    Status order1 =
-        combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()),
-        order2 =
-            combiner.typedReduce(key, Arrays.asList(secondSync, firstSync, newFile).iterator());
-
-    assertEquals(order1, order2);
-  }
-
-  @Test
-  public void commutativeNewFileSingleBuilder() {
-    Status newFile = StatusUtil.fileCreated(100),
-        firstSync = StatusUtil.ingestedUntil(builder, 100),
-        secondSync = StatusUtil.ingestedUntil(builder, 200);
-
-    Status order1 =
-        combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()),
-        order2 =
-            combiner.typedReduce(key, Arrays.asList(secondSync, firstSync, newFile).iterator());
-
-    assertEquals(order1, order2);
-  }
-
-  @Test
-  public void commutativeNewUpdates() {
-    Status newFile = StatusUtil.fileCreated(100), firstSync = StatusUtil.ingestedUntil(100),
-        secondSync = StatusUtil.ingestedUntil(200);
-
-    Status order1 =
-        combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()),
-        order2 =
-            combiner.typedReduce(key, Arrays.asList(newFile, secondSync, firstSync).iterator());
-
-    assertEquals(order1, order2);
-  }
-
-  @Test
-  public void commutativeNewUpdatesSingleBuilder() {
-    Status newFile = StatusUtil.fileCreated(100),
-        firstSync = StatusUtil.ingestedUntil(builder, 100),
-        secondSync = StatusUtil.ingestedUntil(builder, 200);
-
-    Status order1 =
-        combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()),
-        order2 =
-            combiner.typedReduce(key, Arrays.asList(newFile, secondSync, firstSync).iterator());
-
-    assertEquals(order1, order2);
-  }
-
-  @Test
-  public void commutativeWithClose() {
-    Status newFile = StatusUtil.fileCreated(100), closed = StatusUtil.fileClosed(),
-        secondSync = StatusUtil.ingestedUntil(200);
-
-    Status order1 =
-        combiner.typedReduce(key, Arrays.asList(newFile, closed, secondSync).iterator()),
-        order2 = combiner.typedReduce(key, Arrays.asList(newFile, secondSync, closed).iterator());
-
-    assertEquals(order1, order2);
-  }
-
-  @Test
-  public void commutativeWithCloseSingleBuilder() {
-    Status newFile = StatusUtil.fileCreated(100), closed = StatusUtil.fileClosed(),
-        secondSync = StatusUtil.ingestedUntil(builder, 200);
-
-    Status order1 =
-        combiner.typedReduce(key, Arrays.asList(newFile, closed, secondSync).iterator()),
-        order2 = combiner.typedReduce(key, Arrays.asList(newFile, secondSync, closed).iterator());
-
-    assertEquals(order1, order2);
-  }
-
-  @Test
-  public void commutativeWithMultipleUpdates() {
-    Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(100),
-        update2 = StatusUtil.ingestedUntil(200), repl1 = StatusUtil.replicated(50),
-        repl2 = StatusUtil.replicated(150);
-
-    Status order1 = combiner.typedReduce(key,
-        Arrays.asList(newFile, update1, repl1, update2, repl2).iterator());
-
-    // Got all replication updates before ingest updates
-    Status permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, repl1, update1, repl2, update2).iterator());
-
-    assertEquals(order1, permutation);
-
-    // All replications before updates
-    permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, repl1, repl2, update1, update2).iterator());
-
-    assertEquals(order1, permutation);
-
-    // All updates before replications
-    permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, update1, update2, repl1, repl2, update1, update2).iterator());
-
-    assertEquals(order1, permutation);
-  }
-
-  @Test
-  public void commutativeWithMultipleUpdatesSingleBuilder() {
-    Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(builder, 100),
-        update2 = StatusUtil.ingestedUntil(builder, 200),
-        repl1 = StatusUtil.replicated(builder, 50), repl2 = StatusUtil.replicated(builder, 150);
-
-    Status order1 = combiner.typedReduce(key,
-        Arrays.asList(newFile, update1, repl1, update2, repl2).iterator());
-
-    // Got all replication updates before ingest updates
-    Status permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, repl1, update1, repl2, update2).iterator());
-
-    assertEquals(order1, permutation);
-
-    // All replications before updates
-    permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, repl1, repl2, update1, update2).iterator());
-
-    assertEquals(order1, permutation);
-
-    // All updates before replications
-    permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, update1, update2, repl1, repl2).iterator());
-
-    assertEquals(order1, permutation);
-  }
-
-  @Test
-  public void duplicateStatuses() {
-    Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(builder, 100),
-        update2 = StatusUtil.ingestedUntil(builder, 200),
-        repl1 = StatusUtil.replicated(builder, 50), repl2 = StatusUtil.replicated(builder, 150);
-
-    Status order1 = combiner.typedReduce(key,
-        Arrays.asList(newFile, update1, repl1, update2, repl2).iterator());
-
-    // Repeat the same thing more than once
-    Status permutation = combiner.typedReduce(key,
-        Arrays.asList(newFile, repl1, update1, update1, repl2, update2, update2).iterator());
-
-    assertEquals(order1, permutation);
-  }
-
-  @Test
-  public void fileClosedTimePropagated() {
-    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(50).build();
-    Status stat2 =
-        Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).build();
-
-    Status combined = combiner.typedReduce(key, Arrays.asList(stat1, stat2).iterator());
-
-    assertEquals(stat1, combined);
-  }
-
-  @Test
-  public void fileClosedTimeChoosesEarliestIgnoringDefault() {
-    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(50).build();
-    Status stat2 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(100).build();
-
-    Status combined = combiner.typedReduce(key, Arrays.asList(stat1, stat2).iterator());
-
-    assertEquals(stat1, combined);
-
-    Status stat3 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(100).build();
-
-    Status combined2 = combiner.typedReduce(key, Arrays.asList(combined, stat3).iterator());
-
-    assertEquals(combined, combined2);
-  }
-
-  @Test
-  public void testCombination() {
-    List<Status> status = new ArrayList<>();
-    long time = System.currentTimeMillis();
-
-    status.add(StatusUtil.fileCreated(time));
-    status.add(StatusUtil.openWithUnknownLength());
-    status.add(StatusUtil.fileClosed());
-
-    Status combined = combiner.typedReduce(new Key("row"), status.iterator());
-
-    assertEquals(time, combined.getCreatedTime());
-    assertTrue(combined.getInfiniteEnd());
-    assertTrue(combined.getClosed());
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusUtilTest.java
deleted file mode 100644
index cef67b7..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusUtilTest.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.
- */
-package org.apache.accumulo.server.replication;
-
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class StatusUtilTest {
-
-  @Test
-  public void newFileIsNotCompletelyReplicated() {
-    assertFalse(StatusUtil.isSafeForRemoval(StatusUtil.fileCreated(0L)));
-  }
-
-  @Test
-  public void openFileIsNotCompletelyReplicated() {
-    assertFalse(StatusUtil.isSafeForRemoval(Status.newBuilder().setClosed(false).setBegin(0)
-        .setEnd(1000).setInfiniteEnd(false).build()));
-  }
-
-  @Test
-  public void closedFileWithDifferentBeginEndIsNotCompletelyReplicated() {
-    assertFalse(StatusUtil.isSafeForRemoval(Status.newBuilder().setClosed(true).setBegin(0)
-        .setEnd(1000).setInfiniteEnd(false).build()));
-  }
-
-  @Test
-  public void closedFileWithInfEndAndNonMaxBeginIsNotCompletelyReplicated() {
-    assertFalse(StatusUtil.isSafeForRemoval(
-        Status.newBuilder().setClosed(true).setInfiniteEnd(true).setBegin(10000).build()));
-  }
-
-  @Test
-  public void closedFileWithInfEndAndMaxBeginIsCompletelyReplicated() {
-    assertTrue(StatusUtil.isSafeForRemoval(
-        Status.newBuilder().setClosed(true).setInfiniteEnd(true).setBegin(Long.MAX_VALUE).build()));
-  }
-
-  @Test
-  public void closeFileWithEqualBeginEndIsCompletelyReplicated() {
-    assertTrue(StatusUtil.isSafeForRemoval(
-        Status.newBuilder().setClosed(true).setEnd(100000).setBegin(100000).build()));
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/proto/StatusTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/proto/StatusTest.java
deleted file mode 100644
index ee0103e..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/proto/StatusTest.java
+++ /dev/null
@@ -1,39 +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.server.replication.proto;
-
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class StatusTest {
-
-  @Test
-  public void equality() {
-    Status replicated = Status.newBuilder().setBegin(Long.MAX_VALUE).setEnd(0).setInfiniteEnd(true)
-        .setClosed(false).build();
-    Status unreplicated = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true)
-        .setClosed(false).build();
-
-    assertNotEquals(replicated, unreplicated);
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java b/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java
index 4d30ced..f3584d7 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/rpc/SaslServerConnectionParamsTest.java
@@ -80,7 +80,7 @@
       SystemToken token = new SystemToken();
       token.readFields(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
 
-      final SaslConnectionParams saslParams = new SaslServerConnectionParams(rpcConf, token);
+      final SaslConnectionParams saslParams = new SaslServerConnectionParams(rpcConf, token, null);
       assertEquals(primary, saslParams.getKerberosServerPrimary());
       assertEquals(SaslMechanism.GSSAPI, saslParams.getMechanism());
       assertNull(saslParams.getCallbackHandler());
diff --git a/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java
index d8230e8..80d261f 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/rpc/TServerUtilsTest.java
@@ -166,14 +166,13 @@
   public void testStartServerNonDefaultPorts() throws Exception {
     TServer server = null;
 
-    // This test finds 6 free ports in more-or-less a contiguous way and then
+    // This test finds 5 free ports in more-or-less a contiguous way and then
     // uses those port numbers to Accumulo services in the below (ascending) sequence
     // 0. TServer default client port (this test binds to this port to force a port search)
     // 1. GC
     // 2. Manager
     // 3. Monitor
-    // 4. Manager Replication Coordinator
-    // 5. One free port - this is the one that we expect the TServer to finally use
+    // 4. One free port - this is the one that we expect the TServer to finally use
     int[] ports = findTwoFreeSequentialPorts(1024);
     int tserverDefaultPort = ports[0];
     conf.set(Property.TSERV_CLIENTPORT, Integer.toString(tserverDefaultPort));
@@ -187,11 +186,7 @@
     conf.set(Property.MONITOR_PORT, Integer.toString(monitorPort));
 
     ports = findTwoFreeSequentialPorts(monitorPort + 1);
-    int managerReplCoordPort = ports[0];
-    @SuppressWarnings("deprecation")
-    Property p = Property.MANAGER_REPLICATION_COORDINATOR_PORT;
-    conf.set(p, Integer.toString(managerReplCoordPort));
-    int tserverFinalPort = ports[1];
+    int tserverFinalPort = ports[0];
 
     conf.set(Property.TSERV_PORTSEARCH, "true");
 
@@ -205,7 +200,6 @@
     assertTrue(reservedPorts.containsKey(gcPort));
     assertTrue(reservedPorts.containsKey(managerPort));
     assertTrue(reservedPorts.containsKey(monitorPort));
-    assertTrue(reservedPorts.containsKey(managerReplCoordPort));
 
     InetAddress addr = InetAddress.getByName("localhost");
     try (ServerSocket s = new ServerSocket(tserverDefaultPort, 50, addr)) {
@@ -292,7 +286,7 @@
         // keep trying
       }
     }
-    throw new RuntimeException("Unable to find open port");
+    throw new IllegalStateException("Unable to find open port");
   }
 
   private ServerAddress startServer() throws Exception {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
index bc75dc7..ecd7642 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
@@ -86,22 +86,6 @@
     assertTrue(hash.startsWith(SystemToken.SALT_PREFIX));
   }
 
-  /**
-   * This is a test to ensure the SYSTEM_TOKEN_NAME string literal in
-   * {@link org.apache.accumulo.core.clientImpl.ConnectorImpl} is kept up-to-date if we move the
-   * {@link SystemToken}<br>
-   *
-   * @deprecated This check will not be needed after Connector is removed
-   */
-  @Deprecated(since = "2.0.0")
-  @Test
-  public void testSystemToken() {
-    assertEquals("org.apache.accumulo.server.security.SystemCredentials$SystemToken",
-        SystemToken.class.getName());
-    assertEquals(SystemCredentials.get(instanceId, siteConfig).getToken().getClass(),
-        SystemToken.class);
-  }
-
   @Test
   public void testSystemCredentials() {
     Credentials a = SystemCredentials.get(instanceId, siteConfig);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java
index 00d9e6c..7198e16 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java
@@ -214,6 +214,8 @@
     Thread.sleep(50);
 
     // Make a second token for the same user
+    // Briefly sleep to guarantee token is unique, since the token is based on the time
+    Thread.sleep(100);
     Entry<Token<AuthenticationTokenIdentifier>,AuthenticationTokenIdentifier> pair2 =
         secretManager.generateToken(principal, cfg);
     Token<AuthenticationTokenIdentifier> token2 = pair2.getKey();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
index 3f0f743..00af659 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
@@ -30,6 +30,7 @@
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.Arrays;
+import java.util.Base64;
 import java.util.Collections;
 import java.util.Set;
 import java.util.TreeSet;
@@ -51,6 +52,9 @@
 
 public class ZKAuthenticatorTest {
 
+  // test password
+  private byte[] rawPass = "myPassword".getBytes(UTF_8);
+
   @Test
   public void testPermissionIdConversions() {
     for (SystemPermission s : SystemPermission.values()) {
@@ -106,28 +110,25 @@
 
   @Test
   public void testEncryption() throws AccumuloException {
-    byte[] rawPass = "myPassword".getBytes(UTF_8);
     byte[] storedBytes;
 
     storedBytes = ZKSecurityTool.createPass(rawPass.clone());
     assertTrue(ZKSecurityTool.checkCryptPass(rawPass.clone(), storedBytes));
   }
 
-  @Deprecated
   @Test
-  public void testOutdatedEncryption() throws AccumuloException {
-    byte[] rawPass = "myPassword".getBytes();
-    byte[] storedBytes;
+  public void testOutdatedPassword() throws AccumuloException {
+    // hard-coded test password was created using Accumulo's old password SHA-256 hashing,
+    // using "myPassword".getBytes(UTF_8) for the password bytes, and
+    // using the 8 byte salt array = new byte[] {0, 1, 2, 3, 4, 5, 6, 7};
+    byte[] storedBytes =
+        Base64.getDecoder().decode("AAECAwQFBgeD69bFtHx6yzb4j70qBzgNmhR8kTRyJbX3cdzL15jihQ==");
 
-    storedBytes = ZKSecurityTool.createOutdatedPass(rawPass);
-    assertTrue(ZKSecurityTool.checkPass(rawPass, storedBytes));
-  }
-
-  @Test
-  public void testEncryptionDifference() throws AccumuloException {
-    byte[] rawPass = "myPassword".getBytes();
-    @SuppressWarnings("deprecation")
-    byte[] storedBytes = ZKSecurityTool.createOutdatedPass(rawPass);
+    // this test is only checking one typical outdated password; as such, we can't infer much from
+    // the fact that it fails to validate. So, this is mostly checking that it fails with a return
+    // value of "false" rather than throwing an exception; as long as the password check fails
+    // with the return value and doesn't blow things up, the root user can change the user's
+    // password to update it to the new format
     assertFalse(ZKSecurityTool.checkCryptPass(rawPass, storedBytes));
   }
 
@@ -135,7 +136,6 @@
   public void testUserAuthentication() throws Exception {
     // testing the usecase when trying to authenticate with the new hash type
     String principal = "myTestUser";
-    byte[] rawPass = "myPassword".getBytes(UTF_8);
     // creating hash with up to date algorithm
     byte[] newHash = ZKSecurityTool.createPass(rawPass.clone());
 
@@ -161,40 +161,4 @@
     assertTrue(auth.authenticateUser(principal, token));
     verify(context, zr, zk);
   }
-
-  @Test
-  public void testUserAuthenticationUpdate() throws Exception {
-    // testing the usecase when trying to authenticate with the outdated hash type
-    String principal = "myTestUser";
-    byte[] rawPass = "myPassword".getBytes(UTF_8);
-    // creating hash with outdated algorithm
-    @SuppressWarnings("deprecation")
-    byte[] outdatedHash = ZKSecurityTool.createOutdatedPass(rawPass);
-
-    // mocking zk interaction
-    ServerContext context = MockServerContext.getWithZK(InstanceId.of("example"), "", 30_000);
-    ZooReaderWriter zr = createMock(ZooReaderWriter.class);
-    expect(context.getZooReader()).andReturn(zr).anyTimes();
-    expect(context.getZooReaderWriter()).andReturn(zr).anyTimes();
-    ZooKeeper zk = createMock(ZooKeeper.class);
-    expect(zk.getChildren(anyObject(), anyObject())).andReturn(Arrays.asList(principal)).anyTimes();
-    expect(zk.exists(matches("/accumulo/example/users/" + principal), anyObject(Watcher.class)))
-        .andReturn(new Stat()).anyTimes();
-    expect(zr.getZooKeeper()).andReturn(zk).anyTimes();
-    expect(zk.getData(matches("/accumulo/example/users/" + principal), anyObject(), anyObject()))
-        .andReturn(outdatedHash).once();
-    // expecting that the new hash is pushed to zk
-    expect(zr.putPrivatePersistentData(matches("/accumulo/example/users/" + principal), anyObject(),
-        anyObject())).andReturn(true).once();
-    replay(context, zr, zk);
-
-    // creating authenticator
-    ZKAuthenticator auth = new ZKAuthenticator();
-    auth.initialize(context);
-
-    PasswordToken token = new PasswordToken(rawPass.clone());
-    // verifying that if the outdated type of hash is stored in zk authentication works as expected
-    assertTrue(auth.authenticateUser(principal, token));
-    verify(context, zr, zk);
-  }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
index d3f77a3..36388f9 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
@@ -50,12 +50,6 @@
   }
 
   @Test
-  @SuppressWarnings("deprecation")
-  public void testStopMasterCommand() {
-    new Admin.StopMasterCommand();
-  }
-
-  @Test
   public void testStopAllCommand() {
     new Admin.StopAllCommand();
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java
index d38b5a5..a65d776 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java
@@ -20,21 +20,21 @@
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.server.util.FileUtil.FileInfo;
+import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
 public class FileInfoTest {
-  private Key key1;
-  private Key key2;
+  private Text row1;
+  private Text row2;
   private FileInfo info;
 
   @BeforeEach
   public void setUp() {
-    key1 = new Key("row1");
-    key2 = new Key("row2");
-    info = new FileInfo(key1, key2);
+    row1 = new Text("row1");
+    row2 = new Text("row2");
+    info = new FileInfo(row1, row2);
   }
 
   @Test
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/FileSystemMonitorTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/FileSystemMonitorTest.java
deleted file mode 100644
index ead537f..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/util/FileSystemMonitorTest.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.server.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.InputStreamReader;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.server.util.FileSystemMonitor.Mount;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FileSystemMonitorTest {
-  private static final Logger log = LoggerFactory.getLogger(FileSystemMonitorTest.class);
-
-  @Test
-  public void testFilteredMountEntries() throws Exception {
-    String[] mountEntries = {"rootfs / rootfs rw 0 0",
-        "proc /proc proc rw,nosuid,nodev,noexec,relatime 0 0",
-        "sysfs /sys sysfs rw,seclabel,nosuid,nodev,noexec,relatime 0 0",
-        "devtmpfs /dev devtmpfs rw,seclabel,nosuid,size=8119336k,nr_inodes=2029834,mode=755 0 0",
-        "securityfs /sys/kernel/security securityfs rw,nosuid,nodev,noexec,relatime 0 0",
-        "tmpfs /dev/shm tmpfs rw,seclabel,nosuid,nodev 0 0",
-        "devpts /dev/pts devpts rw,seclabel,nosuid,noexec,relatime,gid=5,mode=620,ptmxmode=000 0 0",
-        "tmpfs /run tmpfs rw,seclabel,nosuid,nodev,mode=755 0 0",
-        "tmpfs /sys/fs/cgroup tmpfs ro,seclabel,nosuid,nodev,noexec,mode=755 0 0",
-        "cgroup /sys/fs/cgroup/systemd cgroup rw,nosuid,nodev,noexec,relatime,xattr,"
-            + "release_agent=/usr/lib/systemd/systemd-cgroups-agent,name=systemd 0 0",
-        "pstore /sys/fs/pstore pstore rw,nosuid,nodev,noexec,relatime 0 0",
-        "cgroup /sys/fs/cgroup/cpuset cgroup rw,nosuid,nodev,noexec,relatime,cpuset 0 0",
-        "cgroup /sys/fs/cgroup/cpu,cpuacct cgroup rw,nosuid,nodev,noexec,relatime,cpuacct,cpu 0 0",
-        "cgroup /sys/fs/cgroup/memory cgroup rw,nosuid,nodev,noexec,relatime,memory 0 0",
-        "cgroup /sys/fs/cgroup/devices cgroup rw,nosuid,nodev,noexec,relatime,devices 0 0",
-        "cgroup /sys/fs/cgroup/freezer cgroup rw,nosuid,nodev,noexec,relatime,freezer 0 0",
-        "cgroup /sys/fs/cgroup/net_cls cgroup rw,nosuid,nodev,noexec,relatime,net_cls 0 0",
-        "cgroup /sys/fs/cgroup/blkio cgroup rw,nosuid,nodev,noexec,relatime,blkio 0 0",
-        "cgroup /sys/fs/cgroup/perf_event cgroup rw,nosuid,nodev,noexec,relatime,perf_event 0 0",
-        "cgroup /sys/fs/cgroup/hugetlb cgroup rw,nosuid,nodev,noexec,relatime,hugetlb 0 0",
-        "configfs /sys/kernel/config configfs rw,relatime 0 0",
-        "/dev/vda1 / xfs rw,seclabel,relatime,attr2,inode64,noquota 0 0",
-        "/dev/vda2 /ignoreme reiserfs rw 0 0",
-        "rpc_pipefs /var/lib/nfs/rpc_pipefs rpc_pipefs rw,relatime 0 0",
-        "selinuxfs /sys/fs/selinux selinuxfs rw,relatime 0 0",
-        "systemd-1 /proc/sys/fs/binfmt_misc autofs rw,relatime,fd=32,pgrp=1,"
-            + "timeout=300,minproto=5,maxproto=5,direct 0 0",
-        "debugfs /sys/kernel/debug debugfs rw,relatime 0 0",
-        "mqueue /dev/mqueue mqueue rw,seclabel,relatime 0 0",
-        "hugetlbfs /dev/hugepages hugetlbfs rw,seclabel,relatime 0 0",
-        "sunrpc /proc/fs/nfsd nfsd rw,relatime 0 0",
-        "/dev/vdb /grid/0 ext4 rw,seclabel,relatime,data=ordered 0 0"};
-
-    StringBuilder sb = new StringBuilder(256);
-    for (String mountEntry : mountEntries) {
-      if (sb.length() > 0) {
-        sb.append("\n");
-      }
-      sb.append(mountEntry);
-    }
-
-    ByteArrayInputStream is = new ByteArrayInputStream(sb.toString().getBytes(UTF_8));
-    BufferedReader reader = new BufferedReader(new InputStreamReader(is));
-
-    List<Mount> mounts = FileSystemMonitor.getMountsFromFile(reader);
-    log.info("Filtered mount points: " + mounts);
-    assertEquals(2, mounts.size());
-    Set<String> expectedCheckedMountPoints = new HashSet<>();
-    expectedCheckedMountPoints.add("/");
-    expectedCheckedMountPoints.add("/grid/0");
-    for (Mount mount : mounts) {
-      assertTrue(expectedCheckedMountPoints.remove(mount.mountPoint),
-          "Did not expect to find " + mount);
-    }
-    assertEquals(0, expectedCheckedMountPoints.size(),
-        "Should not have any extra mount points: " + expectedCheckedMountPoints);
-  }
-
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java
index 748dffa..7eb7520 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/ManagerMetadataUtilTest.java
@@ -43,7 +43,9 @@
   @BeforeEach
   public void before() {
     conf = EasyMock.createMock(AccumuloConfiguration.class);
-    EasyMock.expect(conf.get(Property.TSERV_LAST_LOCATION_MODE)).andReturn("assignment");
+    @SuppressWarnings("deprecation")
+    Property DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY = Property.TSERV_LAST_LOCATION_MODE;
+    EasyMock.expect(conf.get(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY)).andReturn("assignment");
     context = EasyMock.createMock(ClientContext.class);
     EasyMock.expect(context.getConfiguration()).andReturn(conf).once();
     tabletMutator = EasyMock.createMock(Ample.TabletMutator.class);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
deleted file mode 100644
index 3684664..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
+++ /dev/null
@@ -1,164 +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.server.util;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.IteratorSetting.Column;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.Writer;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-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.Combiner;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.server.replication.StatusCombiner;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class ReplicationTableUtilTest {
-
-  @Test
-  public void properPathInRow() throws Exception {
-    Writer writer = EasyMock.createNiceMock(Writer.class);
-    writer.update(EasyMock.anyObject(Mutation.class));
-    final List<Mutation> mutations = new ArrayList<>();
-
-    // Mock a Writer to just add the mutation to a list
-    EasyMock.expectLastCall().andAnswer(() -> {
-      mutations.add(((Mutation) EasyMock.getCurrentArguments()[0]));
-      return null;
-    });
-
-    EasyMock.replay(writer);
-
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    ClientContext context = EasyMock.createMock(ClientContext.class);
-    EasyMock.expect(context.getCredentials()).andReturn(creds).anyTimes();
-    EasyMock.replay(context);
-
-    // Magic hook to create a Writer
-    ReplicationTableUtil.addWriter(creds, writer);
-
-    // Example file seen coming out of LogEntry
-    UUID uuid = UUID.randomUUID();
-    String myFile = "file:////home/user/accumulo/wal/server+port/" + uuid;
-
-    long createdTime = System.currentTimeMillis();
-    ReplicationTableUtil.updateFiles(context, new KeyExtent(TableId.of("1"), null, null), myFile,
-        StatusUtil.fileCreated(createdTime));
-
-    verify(writer);
-
-    assertEquals(1, mutations.size());
-    Mutation m = mutations.get(0);
-
-    assertEquals(
-        ReplicationSection.getRowPrefix() + "file:/home/user/accumulo/wal/server+port/" + uuid,
-        new Text(m.getRow()).toString());
-
-    List<ColumnUpdate> updates = m.getUpdates();
-    assertEquals(1, updates.size());
-    ColumnUpdate update = updates.get(0);
-
-    assertEquals(ReplicationSection.COLF, new Text(update.getColumnFamily()));
-    assertEquals("1", new Text(update.getColumnQualifier()).toString());
-    assertEquals(StatusUtil.fileCreatedValue(createdTime), new Value(update.getValue()));
-  }
-
-  @Test
-  public void replEntryMutation() {
-    // We stopped using a WAL -- we need a reference that this WAL needs to be replicated completely
-    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true)
-        .setCreatedTime(System.currentTimeMillis()).build();
-    String file = "file:///accumulo/wal/127.0.0.1+9997" + UUID.randomUUID();
-    Path filePath = new Path(file);
-    Text row = new Text(filePath.toString());
-    KeyExtent extent = new KeyExtent(TableId.of("1"), new Text("b"), new Text("a"));
-
-    Mutation m =
-        ReplicationTableUtil.createUpdateMutation(filePath, ProtobufUtil.toValue(stat), extent);
-
-    assertEquals(new Text(ReplicationSection.getRowPrefix() + row), new Text(m.getRow()));
-    assertEquals(1, m.getUpdates().size());
-    ColumnUpdate col = m.getUpdates().get(0);
-
-    assertEquals(ReplicationSection.COLF, new Text(col.getColumnFamily()));
-    assertEquals(extent.tableId().canonical(), new Text(col.getColumnQualifier()).toString());
-    assertEquals(0, col.getColumnVisibility().length);
-    assertArrayEquals(stat.toByteArray(), col.getValue());
-  }
-
-  @Test
-  public void setsCombinerOnMetadataCorrectly() throws Exception {
-    AccumuloClient client = createMock(AccumuloClient.class);
-    TableOperations tops = createMock(TableOperations.class);
-
-    String myMetadataTable = "mymetadata";
-    Map<String,EnumSet<IteratorScope>> iterators = new HashMap<>();
-    iterators.put("vers", EnumSet.of(IteratorScope.majc, IteratorScope.minc, IteratorScope.scan));
-    IteratorSetting combiner = new IteratorSetting(9, "replcombiner", StatusCombiner.class);
-    Combiner.setColumns(combiner, Collections.singletonList(new Column(ReplicationSection.COLF)));
-
-    expect(client.tableOperations()).andReturn(tops);
-    expect(tops.listIterators(myMetadataTable)).andReturn(iterators);
-    tops.attachIterator(myMetadataTable, combiner);
-    expectLastCall().once();
-
-    expect(tops.getConfiguration(myMetadataTable)).andReturn(Collections.emptyMap());
-    tops.setProperty(myMetadataTable, Property.TABLE_FORMATTER_CLASS.getKey(),
-        ReplicationTableUtil.STATUS_FORMATTER_CLASS_NAME);
-    expectLastCall().once();
-
-    replay(client, tops);
-
-    ReplicationTableUtil.configureMetadataTable(client, myMetadataTable);
-
-    verify(client, tops);
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TableDiskUsageTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TableDiskUsageTest.java
index a275424..009fcc1 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TableDiskUsageTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TableDiskUsageTest.java
@@ -25,6 +25,7 @@
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -36,9 +37,8 @@
 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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
@@ -66,8 +66,8 @@
 
   @BeforeAll
   public static void beforeClass() {
-    tableIdToNameMap.put(RootTable.ID, MetadataTable.NAME);
-    tableIdToNameMap.put(MetadataTable.ID, MetadataTable.NAME);
+    tableIdToNameMap.put(AccumuloTable.ROOT.tableId(), AccumuloTable.METADATA.tableName());
+    tableIdToNameMap.put(AccumuloTable.METADATA.tableId(), AccumuloTable.METADATA.tableName());
     tableIdToNameMap.put(tableId1, "table1");
     tableIdToNameMap.put(tableId2, "table2");
     tableIdToNameMap.put(tableId3, "table3");
@@ -135,20 +135,21 @@
     final Scanner scanner = EasyMock.createMock(Scanner.class);
 
     // Expect root table instead to be scanned
-    EasyMock.expect(client.createScanner(RootTable.NAME, Authorizations.EMPTY)).andReturn(scanner);
+    EasyMock.expect(client.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY))
+        .andReturn(scanner);
     EasyMock.expect(client.getTableIdToNameMap()).andReturn(tableIdToNameMap);
 
     Map<Key,Value> tableEntries = new HashMap<>();
-    appendFileMetadata(tableEntries,
-        getTabletFile(MetadataTable.ID, MetadataTable.NAME, "C0001.rf"), 1024);
-    mockTableScan(scanner, tableEntries, MetadataTable.ID);
+    appendFileMetadata(tableEntries, getTabletFile(AccumuloTable.METADATA.tableId(),
+        AccumuloTable.METADATA.tableName(), "C0001.rf"), 1024);
+    mockTableScan(scanner, tableEntries, AccumuloTable.METADATA.tableId());
 
     EasyMock.replay(client, scanner);
 
     Map<SortedSet<String>,Long> result =
-        TableDiskUsage.getDiskUsage(tableSet(MetadataTable.ID), client);
+        TableDiskUsage.getDiskUsage(tableSet(AccumuloTable.METADATA.tableId()), client);
 
-    assertEquals(1024, getTotalUsage(result, MetadataTable.ID));
+    assertEquals(1024, getTotalUsage(result, AccumuloTable.METADATA.tableId()));
     assertEquals(1, result.size());
     Map.Entry<SortedSet<String>,Long> firstResult =
         result.entrySet().stream().findFirst().orElseThrow();
@@ -268,33 +269,34 @@
 
   private static Long getTotalUsage(Map<SortedSet<String>,Long> result, TableId tableId) {
     return result.entrySet().stream()
-        .filter(entry -> entry.getKey().contains(getTableName(tableId)))
-        .mapToLong(entry -> entry.getValue()).sum();
+        .filter(entry -> entry.getKey().contains(getTableName(tableId))).mapToLong(Entry::getValue)
+        .sum();
   }
 
   private static String getTableName(TableId tableId) {
     return tableIdToNameMap.get(tableId);
   }
 
-  private static void appendFileMetadata(Map<Key,Value> tableEntries, TabletFile file, long size) {
-    tableEntries.put(
-        new Key(new Text(file.getTableId() + "<"),
-            MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, file.getMetaInsertText()),
+  private static void appendFileMetadata(Map<Key,Value> tableEntries, ReferencedTabletFile file,
+      long size) {
+    tableEntries.put(new Key(new Text(file.getTableId() + "<"),
+        MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, file.insert().getMetadataText()),
         new DataFileValue(size, 1).encodeAsValue());
   }
 
-  private static TabletFile getTabletFile(String volume, TableId tableId, String tablet,
+  private static ReferencedTabletFile getTabletFile(String volume, TableId tableId, String tablet,
       String fileName) {
-    return new TabletFile(new Path(
+    return new ReferencedTabletFile(new Path(
         volume + Constants.HDFS_TABLES_DIR + "/" + tableId + "/" + tablet + "/" + fileName));
   }
 
-  private static TabletFile getTabletFile(TableId tableId, String tablet, String fileName) {
+  private static ReferencedTabletFile getTabletFile(TableId tableId, String tablet,
+      String fileName) {
     return getTabletFile(volume1, tableId, tablet, fileName);
   }
 
   private void mockScan(ServerContext client, Scanner scanner, int times) throws Exception {
-    EasyMock.expect(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY))
+    EasyMock.expect(client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY))
         .andReturn(scanner).times(times);
     EasyMock.expect(client.getTableIdToNameMap()).andReturn(tableIdToNameMap);
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java
index 17d0068..40cc955 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/fateCommand/SummaryReportTest.java
@@ -23,8 +23,8 @@
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
 import java.util.Map;
@@ -45,8 +45,9 @@
   public void blankReport() {
     Map<String,String> idMap = Map.of("1", "ns1", "2", "tbl1");
     FateSummaryReport report = new FateSummaryReport(idMap, null);
+
     assertNotNull(report);
-    assertTrue(report.getReportTime() != 0);
+    assertNotEquals(0, report.getReportTime());
     assertEquals(Map.of(), report.getStatusCounts());
     assertEquals(Map.of(), report.getCmdCounts());
     assertEquals(Map.of(), report.getStepCounts());
@@ -54,6 +55,7 @@
     assertEquals(Set.of(), report.getStatusFilterNames());
     assertNotNull(report.toJson());
     assertNotNull(report.formatLines());
+
     log.info("json: {}", report.toJson());
     log.info("formatted: {}", report.formatLines());
   }
@@ -78,7 +80,7 @@
     report.gatherTxnStatus(status1);
 
     assertNotNull(report);
-    assertTrue(report.getReportTime() != 0);
+    assertNotEquals(0, report.getReportTime());
     assertEquals(Map.of("IN_PROGRESS", 1), report.getStatusCounts());
     assertEquals(Map.of("?", 1), report.getCmdCounts());
     assertEquals(Map.of("?", 1), report.getStepCounts());
diff --git a/server/compaction-coordinator/pom.xml b/server/compaction-coordinator/pom.xml
index 151cc8f..8531128 100644
--- a/server/compaction-coordinator/pom.xml
+++ b/server/compaction-coordinator/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-compaction-coordinator</artifactId>
@@ -82,29 +82,9 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.easymock</groupId>
-      <artifactId>easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.junit.jupiter</groupId>
       <artifactId>junit-jupiter-api</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.junit.vintage</groupId>
-      <artifactId>junit-vintage-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.powermock</groupId>
-      <artifactId>powermock-api-easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.powermock</groupId>
-      <artifactId>powermock-module-junit4</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
 </project>
diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
index 9ac4ddd..708e674 100644
--- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
+++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
@@ -18,11 +18,12 @@
  */
 package org.apache.accumulo.coordinator;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 import java.net.UnknownHostException;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -30,6 +31,7 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -37,9 +39,11 @@
 
 import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
@@ -54,8 +58,10 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+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.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
@@ -67,18 +73,14 @@
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 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.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.compaction.RunningCompaction;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.rpc.ServerAddress;
@@ -95,6 +97,8 @@
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.google.common.collect.Sets;
+import com.google.common.net.HostAndPort;
+import com.google.common.util.concurrent.Uninterruptibles;
 
 public class CompactionCoordinator extends AbstractServer
     implements CompactionCoordinatorService.Iface, LiveTServerSet.Listener {
@@ -122,7 +126,6 @@
   /* Map of queue name to last time compactor called to get a compaction job */
   private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
 
-  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
   protected SecurityOperation security;
   protected final AccumuloConfiguration aconf;
   protected CompactionFinalizer compactionFinalizer;
@@ -133,20 +136,22 @@
   // Exposed for tests
   protected volatile Boolean shutdown = false;
 
-  private ScheduledThreadPoolExecutor schedExecutor;
+  private final ScheduledThreadPoolExecutor schedExecutor;
+  private final ExecutorService summariesExecutor;
 
-  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+  protected CompactionCoordinator(ConfigOpts opts, String[] args) {
     this(opts, args, null);
   }
 
-  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+  protected CompactionCoordinator(ConfigOpts opts, String[] args, AccumuloConfiguration conf) {
     super("compaction-coordinator", opts, args);
     aconf = conf == null ? super.getConfiguration() : conf;
     schedExecutor = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);
+    summariesExecutor = ThreadPools.getServerThreadPools()
+        .getPoolBuilder("Compaction Summary Gatherer").numCoreThreads(10).build();
     compactionFinalizer = createCompactionFinalizer(schedExecutor);
     tserverSet = createLiveTServerSet();
     setupSecurity();
-    startGCLogger(schedExecutor);
     printStartupMsg();
     startCompactionCleaner(schedExecutor);
     startRunningCleaner(schedExecutor);
@@ -170,13 +175,6 @@
     security = getContext().getSecurityOperation();
   }
 
-  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
-    ScheduledFuture<?> future =
-        schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
-            TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
-    ThreadPools.watchNonCriticalScheduledTask(future);
-  }
-
   protected void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
     ScheduledFuture<?> future =
         schedExecutor.scheduleWithFixedDelay(this::cleanUpCompactors, 0, 5, TimeUnit.MINUTES);
@@ -214,7 +212,8 @@
       CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
       coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
           ServiceLock.path(lockPath), zooLockUUID);
-      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes(UTF_8));
+      coordinatorLock.lock(coordinatorLockWatcher,
+          new ServiceLockData(zooLockUUID, coordinatorClientAddress, ThriftService.COORDINATOR));
 
       coordinatorLockWatcher.waitForChange();
       if (coordinatorLockWatcher.isAcquiredLock()) {
@@ -257,7 +256,7 @@
     try {
       coordinatorAddress = startCoordinatorClientService();
     } catch (UnknownHostException e1) {
-      throw new RuntimeException("Failed to start the coordinator service", e1);
+      throw new IllegalStateException("Failed to start the coordinator service", e1);
     }
     final HostAndPort clientAddress = coordinatorAddress.address;
 
@@ -269,6 +268,7 @@
 
     MetricsInfo metricsInfo = getContext().getMetricsInfo();
     metricsInfo.addServiceTags(getApplicationName(), clientAddress);
+    metricsInfo.addMetricsProducers(this);
     metricsInfo.init();
 
     // On a re-start of the coordinator it's possible that external compactions are in-progress.
@@ -302,7 +302,7 @@
 
       long now = System.currentTimeMillis();
 
-      Map<String,List<HostAndPort>> idleCompactors = getIdleCompactors();
+      Map<String,Set<HostAndPort>> idleCompactors = getIdleCompactors();
       TIME_COMPACTOR_LAST_CHECKED.forEach((queue, lastCheckTime) -> {
         if ((now - lastCheckTime) > getMissingCompactorWarningTime()
             && QUEUE_SUMMARIES.isCompactionsQueued(queue) && idleCompactors.containsKey(queue)) {
@@ -319,19 +319,20 @@
       }
     }
 
+    summariesExecutor.shutdownNow();
     LOG.info("Shutting down");
   }
 
-  private Map<String,List<HostAndPort>> getIdleCompactors() {
+  private Map<String,Set<HostAndPort>> getIdleCompactors() {
 
-    Map<String,List<HostAndPort>> allCompactors =
+    Map<String,Set<HostAndPort>> allCompactors =
         ExternalCompactionUtil.getCompactorAddrs(getContext());
 
     Set<String> emptyQueues = new HashSet<>();
 
     // Remove all of the compactors that are running a compaction
     RUNNING_CACHE.values().forEach(rc -> {
-      List<HostAndPort> busyCompactors = allCompactors.get(rc.getQueueName());
+      Set<HostAndPort> busyCompactors = allCompactors.get(rc.getQueueName());
       if (busyCompactors != null
           && busyCompactors.remove(HostAndPort.fromString(rc.getCompactorAddress()))) {
         if (busyCompactors.isEmpty()) {
@@ -345,40 +346,39 @@
   }
 
   private void updateSummaries() {
-    ExecutorService executor = ThreadPools.getServerThreadPools()
-        .getPoolBuilder("Compaction Summary Gatherer").numCoreThreads(10).build();
-    try {
-      Set<String> queuesSeen = new ConcurrentSkipListSet<>();
 
-      tserverSet.getCurrentServers().forEach(tsi -> {
-        executor.execute(() -> updateSummaries(tsi, queuesSeen));
-      });
+    final ArrayList<Future<?>> tasks = new ArrayList<>();
+    Set<String> queuesSeen = new ConcurrentSkipListSet<>();
 
-      executor.shutdown();
+    tserverSet.getCurrentServers().forEach(tsi -> {
+      tasks.add(summariesExecutor.submit(() -> updateSummaries(tsi, queuesSeen)));
+    });
 
-      try {
-        while (!executor.awaitTermination(1, TimeUnit.MINUTES)) {}
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(e);
+    // Wait for all tasks to complete
+    while (!tasks.isEmpty()) {
+      Iterator<Future<?>> iter = tasks.iterator();
+      while (iter.hasNext()) {
+        Future<?> f = iter.next();
+        if (f.isDone()) {
+          iter.remove();
+        }
       }
-
-      // remove any queues that were seen in the past, but were not seen in the latest gathering of
-      // summaries
-      TIME_COMPACTOR_LAST_CHECKED.keySet().retainAll(queuesSeen);
-
-      // add any queues that were never seen before
-      queuesSeen.forEach(q -> {
-        TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(q, k -> System.currentTimeMillis());
-      });
-    } finally {
-      executor.shutdownNow();
+      Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
     }
+
+    // remove any queues that were seen in the past, but were not seen in the latest gathering of
+    // summaries
+    TIME_COMPACTOR_LAST_CHECKED.keySet().retainAll(queuesSeen);
+
+    // add any queues that were never seen before
+    queuesSeen.forEach(q -> {
+      TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(q, k -> System.currentTimeMillis());
+    });
   }
 
   private void updateSummaries(TServerInstance tsi, Set<String> queuesSeen) {
     try {
-      TabletClientService.Client client = null;
+      TabletServerClientService.Client client = null;
       try {
         LOG.debug("Contacting tablet server {} to get external compaction summaries",
             tsi.getHostPort());
@@ -461,7 +461,7 @@
 
       LOG.trace("Getting compaction for queue {} from tserver {}", queue, tserver.getHostAndPort());
       // Get a compaction from the tserver
-      TabletClientService.Client client = null;
+      TabletServerClientService.Client client = null;
       try {
         client = getTabletServerConnection(tserver);
         TExternalCompactionJob job =
@@ -510,7 +510,7 @@
    * @return thrift client
    * @throws TTransportException thrift error
    */
-  protected TabletClientService.Client getTabletServerConnection(TServerInstance tserver)
+  protected TabletServerClientService.Client getTabletServerConnection(TServerInstance tserver)
       throws TTransportException {
     TServerConnection connection = tserverSet.getConnection(tserver);
     ServerContext serverContext = getContext();
@@ -750,12 +750,12 @@
       LOG.warn("Failed to clean up compactors", e);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
   public static void main(String[] args) throws Exception {
-    try (CompactionCoordinator compactor = new CompactionCoordinator(new ServerOpts(), args)) {
+    try (CompactionCoordinator compactor = new CompactionCoordinator(new ConfigOpts(), args)) {
       compactor.runServer();
     }
   }
diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java
index 45b6161..a781ef4 100644
--- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java
+++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionFinalizer.java
@@ -27,6 +27,7 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutionException;
@@ -48,7 +49,7 @@
 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.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
@@ -122,7 +123,7 @@
 
   private void notifyTserver(Location loc, ExternalCompactionFinalState ecfs) {
 
-    TabletClientService.Client client = null;
+    Client client = null;
     try {
       client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, loc.getHostAndPort(), context);
       if (ecfs.getFinalState() == FinalState.FINISHED) {
@@ -159,8 +160,9 @@
 
         Map<KeyExtent,TabletMetadata> tabletsMetadata;
         var extents = batch.stream().map(ExternalCompactionFinalState::getExtent).collect(toList());
-        try (TabletsMetadata tablets = context.getAmple().readTablets().forTablets(extents)
-            .fetch(ColumnType.LOCATION, ColumnType.PREV_ROW, ColumnType.ECOMP).build()) {
+        try (TabletsMetadata tablets =
+            context.getAmple().readTablets().forTablets(extents, Optional.empty())
+                .fetch(ColumnType.LOCATION, ColumnType.PREV_ROW, ColumnType.ECOMP).build()) {
           tabletsMetadata = tablets.stream().collect(toMap(TabletMetadata::getExtent, identity()));
         }
 
@@ -204,7 +206,7 @@
 
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       } catch (RuntimeException e) {
         LOG.warn("Failed to process pending notifications", e);
       }
@@ -223,7 +225,7 @@
       }
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } catch (RuntimeException e) {
       LOG.warn("Failed to notify tservers", e);
     }
diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CoordinatorLockWatcher.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CoordinatorLockWatcher.java
index 663ecaf4..bbbe731 100644
--- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CoordinatorLockWatcher.java
+++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CoordinatorLockWatcher.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.coordinator;
 
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
 import org.apache.accumulo.core.util.Halt;
 import org.apache.zookeeper.KeeperException.NoAuthException;
 import org.slf4j.Logger;
diff --git a/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java b/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java
deleted file mode 100644
index 1a97e3d..0000000
--- a/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java
+++ /dev/null
@@ -1,691 +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.coordinator;
-
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.expect;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
-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.conf.SiteConfiguration;
-import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
-import org.apache.accumulo.core.metrics.MetricsInfo;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
-import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
-import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-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.compaction.ExternalCompactionUtil;
-import org.apache.accumulo.core.util.compaction.RunningCompaction;
-import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
-import org.apache.accumulo.server.manager.LiveTServerSet;
-import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.accumulo.server.security.AuditedSecurityOperation;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.zookeeper.KeeperException;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.api.easymock.PowerMock;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import com.google.common.collect.Sets;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({CompactionCoordinator.class, DeadCompactionDetector.class, ThriftUtil.class,
-    ExternalCompactionUtil.class})
-@SuppressStaticInitializationFor({"org.apache.log4j.LogManager"})
-@PowerMockIgnore({"org.slf4j.*", "org.apache.logging.*", "org.apache.log4j.*",
-    "org.apache.commons.logging.*", "org.xml.*", "javax.xml.*", "org.w3c.dom.*",
-    "com.sun.org.apache.xerces.*"})
-public class CompactionCoordinatorTest {
-
-  public class TestCoordinator extends CompactionCoordinator {
-
-    private final ServerContext context;
-    private final ServerAddress client;
-    private final TabletClientService.Client tabletServerClient;
-
-    private Set<ExternalCompactionId> metadataCompactionIds = null;
-
-    protected TestCoordinator(CompactionFinalizer finalizer, LiveTServerSet tservers,
-        ServerAddress client, TabletClientService.Client tabletServerClient, ServerContext context,
-        AuditedSecurityOperation security) {
-      super(new ServerOpts(), new String[] {}, context.getConfiguration());
-      this.compactionFinalizer = finalizer;
-      this.tserverSet = tservers;
-      this.client = client;
-      this.tabletServerClient = tabletServerClient;
-      this.context = context;
-      this.security = security;
-    }
-
-    @Override
-    protected void startDeadCompactionDetector() {}
-
-    @Override
-    protected long getTServerCheckInterval() {
-      this.shutdown = true;
-      return 0L;
-    }
-
-    @Override
-    protected void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {}
-
-    @Override
-    protected CompactionFinalizer createCompactionFinalizer(ScheduledThreadPoolExecutor stpe) {
-      return null;
-    }
-
-    @Override
-    protected LiveTServerSet createLiveTServerSet() {
-      return null;
-    }
-
-    @Override
-    protected void setupSecurity() {}
-
-    @Override
-    protected void startGCLogger(ScheduledThreadPoolExecutor stpe) {}
-
-    @Override
-    protected void printStartupMsg() {}
-
-    @Override
-    public ServerContext getContext() {
-      return this.context;
-    }
-
-    @Override
-    protected void getCoordinatorLock(HostAndPort clientAddress)
-        throws KeeperException, InterruptedException {}
-
-    @Override
-    protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
-      return client;
-    }
-
-    @Override
-    protected Client getTabletServerConnection(TServerInstance tserver) throws TTransportException {
-      return tabletServerClient;
-    }
-
-    @Override
-    public void compactionCompleted(TInfo tinfo, TCredentials credentials,
-        String externalCompactionId, TKeyExtent textent, TCompactionStats stats)
-        throws ThriftSecurityException {}
-
-    @Override
-    public void compactionFailed(TInfo tinfo, TCredentials credentials, String externalCompactionId,
-        TKeyExtent extent) throws ThriftSecurityException {}
-
-    void setMetadataCompactionIds(Set<ExternalCompactionId> mci) {
-      metadataCompactionIds = mci;
-    }
-
-    @Override
-    protected Set<ExternalCompactionId> readExternalCompactionIds() {
-      if (metadataCompactionIds == null) {
-        return RUNNING_CACHE.keySet();
-      } else {
-        return metadataCompactionIds;
-      }
-    }
-
-    public Map<String,TreeMap<Short,TreeSet<TServerInstance>>> getQueues() {
-      return CompactionCoordinator.QUEUE_SUMMARIES.QUEUES;
-    }
-
-    public Map<TServerInstance,Set<QueueAndPriority>> getIndex() {
-      return CompactionCoordinator.QUEUE_SUMMARIES.INDEX;
-    }
-
-    public Map<ExternalCompactionId,RunningCompaction> getRunning() {
-      return RUNNING_CACHE;
-    }
-
-    public void resetInternals() {
-      getQueues().clear();
-      getIndex().clear();
-      getRunning().clear();
-      metadataCompactionIds = null;
-    }
-
-  }
-
-  @Test
-  public void testCoordinatorWarningTime() {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-
-    SiteConfiguration aconf = SiteConfiguration.empty()
-        .withOverrides(Map.of(Property.COMPACTOR_MAX_JOB_WAIT_TIME.getKey(), "15s")).build();
-    ConfigurationCopy config = new ConfigurationCopy(aconf);
-    expect(context.getConfiguration()).andReturn(config).anyTimes();
-
-    PowerMock.replay(context);
-
-    var coordinator = new TestCoordinator(null, null, null, null, context, null);
-    // Should be equal to 3 * 15_000 milliseconds
-    assertEquals(45_000, coordinator.getMissingCompactorWarningTime());
-    coordinator.close();
-  }
-
-  @Test
-  public void testCoordinatorColdStartNoCompactions() throws Exception {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-    PowerMock.suppress(PowerMock.methods(ThriftUtil.class, "returnClient"));
-    PowerMock.suppress(PowerMock.methods(DeadCompactionDetector.class, "detectDeadCompactions",
-        "detectDanglingFinalStateMarkers"));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    PowerMock.mockStatic(ExternalCompactionUtil.class);
-    List<RunningCompaction> runningCompactions = new ArrayList<>();
-    expect(ExternalCompactionUtil.getCompactionsRunningOnCompactors(context))
-        .andReturn(runningCompactions);
-    expect(ExternalCompactionUtil.getCompactorAddrs(context)).andReturn(Map.of()).anyTimes();
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-    expect(tservers.getCurrentServers()).andReturn(Collections.emptySet()).anyTimes();
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    TServerInstance tsi = PowerMock.createNiceMock(TServerInstance.class);
-    expect(tsi.getHostPort()).andReturn("localhost:9997").anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-    expect(tsc.getCompactionQueueInfo(anyObject(), anyObject())).andReturn(Collections.emptyList())
-        .anyTimes();
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-
-    PowerMock.replayAll();
-
-    var coordinator = new TestCoordinator(finalizer, tservers, client, tsc, context, security);
-    coordinator.resetInternals();
-    assertEquals(0, coordinator.getQueues().size());
-    assertEquals(0, coordinator.getIndex().size());
-    assertEquals(0, coordinator.getRunning().size());
-    coordinator.run();
-    assertEquals(0, coordinator.getQueues().size());
-    assertEquals(0, coordinator.getIndex().size());
-    assertEquals(0, coordinator.getRunning().size());
-
-    PowerMock.verifyAll();
-    coordinator.resetInternals();
-    coordinator.close();
-  }
-
-  @Test
-  public void testCoordinatorColdStart() throws Exception {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-    PowerMock.suppress(PowerMock.methods(ThriftUtil.class, "returnClient"));
-    PowerMock.suppress(PowerMock.methods(DeadCompactionDetector.class, "detectDeadCompactions",
-        "detectDanglingFinalStateMarkers"));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    TCredentials creds = PowerMock.createNiceMock(TCredentials.class);
-    expect(context.rpcCreds()).andReturn(creds);
-
-    PowerMock.mockStatic(ExternalCompactionUtil.class);
-    List<RunningCompaction> runningCompactions = new ArrayList<>();
-    expect(ExternalCompactionUtil.getCompactionsRunningOnCompactors(context))
-        .andReturn(runningCompactions);
-    expect(ExternalCompactionUtil.getCompactorAddrs(context)).andReturn(Map.of()).anyTimes();
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-    TServerInstance instance = PowerMock.createNiceMock(TServerInstance.class);
-    expect(tservers.getCurrentServers()).andReturn(Collections.singleton(instance)).once();
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    TServerInstance tsi = PowerMock.createNiceMock(TServerInstance.class);
-    expect(tsi.getHostPort()).andReturn("localhost:9997").anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-    TCompactionQueueSummary queueSummary = PowerMock.createNiceMock(TCompactionQueueSummary.class);
-    expect(tsc.getCompactionQueueInfo(anyObject(), anyObject()))
-        .andReturn(Collections.singletonList(queueSummary)).anyTimes();
-    expect(queueSummary.getQueue()).andReturn("R2DQ").anyTimes();
-    expect(queueSummary.getPriority()).andReturn((short) 1).anyTimes();
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-
-    PowerMock.replayAll();
-
-    var coordinator = new TestCoordinator(finalizer, tservers, client, tsc, context, security);
-    coordinator.resetInternals();
-    assertEquals(0, coordinator.getQueues().size());
-    assertEquals(0, coordinator.getIndex().size());
-    assertEquals(0, coordinator.getRunning().size());
-    coordinator.run();
-    assertEquals(1, coordinator.getQueues().size());
-    QueueAndPriority qp = QueueAndPriority.get("R2DQ".intern(), (short) 1);
-    Map<Short,TreeSet<TServerInstance>> m = coordinator.getQueues().get("R2DQ".intern());
-    assertNotNull(m);
-    assertEquals(1, m.size());
-    assertTrue(m.containsKey((short) 1));
-    Set<TServerInstance> t = m.get((short) 1);
-    assertNotNull(t);
-    assertEquals(1, t.size());
-    TServerInstance queuedTsi = t.iterator().next();
-    assertEquals(tsi.getHostPortSession(), queuedTsi.getHostPortSession());
-    assertEquals(1, coordinator.getIndex().size());
-    assertTrue(coordinator.getIndex().containsKey(queuedTsi));
-    Set<QueueAndPriority> i = coordinator.getIndex().get(queuedTsi);
-    assertEquals(1, i.size());
-    assertEquals(qp, i.iterator().next());
-    assertEquals(0, coordinator.getRunning().size());
-
-    PowerMock.verifyAll();
-    coordinator.resetInternals();
-    coordinator.close();
-  }
-
-  @Test
-  public void testCoordinatorRestartNoRunningCompactions() throws Exception {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-    PowerMock.suppress(PowerMock.methods(ThriftUtil.class, "returnClient"));
-    PowerMock.suppress(PowerMock.methods(DeadCompactionDetector.class, "detectDeadCompactions",
-        "detectDanglingFinalStateMarkers"));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    TCredentials creds = PowerMock.createNiceMock(TCredentials.class);
-    expect(context.rpcCreds()).andReturn(creds);
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-    TServerInstance instance = PowerMock.createNiceMock(TServerInstance.class);
-    HostAndPort tserverAddress = HostAndPort.fromString("localhost:9997");
-    expect(instance.getHostAndPort()).andReturn(tserverAddress).anyTimes();
-    expect(tservers.getCurrentServers()).andReturn(Sets.newHashSet(instance)).once();
-    tservers.startListeningForTabletServerChanges();
-
-    PowerMock.mockStatic(ExternalCompactionUtil.class);
-    List<RunningCompaction> runningCompactions = new ArrayList<>();
-    expect(ExternalCompactionUtil.getCompactionsRunningOnCompactors(context))
-        .andReturn(runningCompactions);
-    expect(ExternalCompactionUtil.getCompactorAddrs(context)).andReturn(Map.of()).anyTimes();
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    expect(instance.getHostPort()).andReturn("localhost:9997").anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-    TCompactionQueueSummary queueSummary = PowerMock.createNiceMock(TCompactionQueueSummary.class);
-    expect(tsc.getCompactionQueueInfo(anyObject(), anyObject()))
-        .andReturn(Collections.singletonList(queueSummary)).anyTimes();
-    expect(queueSummary.getQueue()).andReturn("R2DQ").anyTimes();
-    expect(queueSummary.getPriority()).andReturn((short) 1).anyTimes();
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-
-    PowerMock.replayAll();
-
-    var coordinator = new TestCoordinator(finalizer, tservers, client, tsc, context, security);
-    coordinator.resetInternals();
-    assertEquals(0, coordinator.getQueues().size());
-    assertEquals(0, coordinator.getIndex().size());
-    assertEquals(0, coordinator.getRunning().size());
-    coordinator.run();
-    assertEquals(1, coordinator.getQueues().size());
-    QueueAndPriority qp = QueueAndPriority.get("R2DQ".intern(), (short) 1);
-    Map<Short,TreeSet<TServerInstance>> m = coordinator.getQueues().get("R2DQ".intern());
-    assertNotNull(m);
-    assertEquals(1, m.size());
-    assertTrue(m.containsKey((short) 1));
-    Set<TServerInstance> t = m.get((short) 1);
-    assertNotNull(t);
-    assertEquals(1, t.size());
-    TServerInstance queuedTsi = t.iterator().next();
-    assertEquals(instance.getHostPortSession(), queuedTsi.getHostPortSession());
-    assertEquals(1, coordinator.getIndex().size());
-    assertTrue(coordinator.getIndex().containsKey(queuedTsi));
-    Set<QueueAndPriority> i = coordinator.getIndex().get(queuedTsi);
-    assertEquals(1, i.size());
-    assertEquals(qp, i.iterator().next());
-    assertEquals(0, coordinator.getRunning().size());
-
-    PowerMock.verifyAll();
-    coordinator.resetInternals();
-    coordinator.close();
-  }
-
-  @Test
-  public void testCoordinatorRestartOneRunningCompaction() throws Exception {
-
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-    PowerMock.suppress(PowerMock.methods(ThriftUtil.class, "returnClient"));
-    PowerMock.suppress(PowerMock.methods(DeadCompactionDetector.class, "detectDeadCompactions",
-        "detectDanglingFinalStateMarkers"));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    TCredentials creds = PowerMock.createNiceMock(TCredentials.class);
-    expect(context.rpcCreds()).andReturn(creds);
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-    TServerInstance instance = PowerMock.createNiceMock(TServerInstance.class);
-    HostAndPort tserverAddress = HostAndPort.fromString("localhost:9997");
-    expect(instance.getHostAndPort()).andReturn(tserverAddress).anyTimes();
-    expect(tservers.getCurrentServers()).andReturn(Sets.newHashSet(instance)).once();
-    tservers.startListeningForTabletServerChanges();
-
-    PowerMock.mockStatic(ExternalCompactionUtil.class);
-    List<RunningCompaction> runningCompactions = new ArrayList<>();
-    ExternalCompactionId eci = ExternalCompactionId.generate(UUID.randomUUID());
-    TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class);
-    expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes();
-    TKeyExtent extent = new TKeyExtent();
-    extent.setTable("1".getBytes());
-    runningCompactions.add(new RunningCompaction(job, tserverAddress.toString(), "queue"));
-    expect(ExternalCompactionUtil.getCompactionsRunningOnCompactors(context))
-        .andReturn(runningCompactions);
-    expect(ExternalCompactionUtil.getCompactorAddrs(context)).andReturn(Map.of()).anyTimes();
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    expect(instance.getHostPort()).andReturn("localhost:9997").anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-    TCompactionQueueSummary queueSummary = PowerMock.createNiceMock(TCompactionQueueSummary.class);
-    expect(tsc.getCompactionQueueInfo(anyObject(), anyObject()))
-        .andReturn(Collections.singletonList(queueSummary)).anyTimes();
-    expect(queueSummary.getQueue()).andReturn("R2DQ").anyTimes();
-    expect(queueSummary.getPriority()).andReturn((short) 1).anyTimes();
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-
-    PowerMock.replayAll();
-
-    var coordinator = new TestCoordinator(finalizer, tservers, client, tsc, context, security);
-    coordinator.resetInternals();
-    assertEquals(0, coordinator.getQueues().size());
-    assertEquals(0, coordinator.getIndex().size());
-    assertEquals(0, coordinator.getRunning().size());
-    coordinator.run();
-    assertEquals(1, coordinator.getQueues().size());
-    QueueAndPriority qp = QueueAndPriority.get("R2DQ".intern(), (short) 1);
-    Map<Short,TreeSet<TServerInstance>> m = coordinator.getQueues().get("R2DQ".intern());
-    assertNotNull(m);
-    assertEquals(1, m.size());
-    assertTrue(m.containsKey((short) 1));
-    Set<TServerInstance> t = m.get((short) 1);
-    assertNotNull(t);
-    assertEquals(1, t.size());
-    TServerInstance queuedTsi = t.iterator().next();
-    assertEquals(instance.getHostPortSession(), queuedTsi.getHostPortSession());
-    assertEquals(1, coordinator.getIndex().size());
-    assertTrue(coordinator.getIndex().containsKey(queuedTsi));
-    Set<QueueAndPriority> i = coordinator.getIndex().get(queuedTsi);
-    assertEquals(1, i.size());
-    assertEquals(qp, i.iterator().next());
-    assertEquals(1, coordinator.getRunning().size());
-
-    PowerMock.verifyAll();
-    coordinator.resetInternals();
-    coordinator.close();
-  }
-
-  @Test
-  public void testGetCompactionJob() throws Exception {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-    PowerMock.suppress(PowerMock.methods(ThriftUtil.class, "returnClient"));
-    PowerMock.suppress(PowerMock.methods(DeadCompactionDetector.class, "detectDeadCompactions",
-        "detectDanglingFinalStateMarkers"));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    TCredentials creds = PowerMock.createNiceMock(TCredentials.class);
-    expect(context.rpcCreds()).andReturn(creds).anyTimes();
-
-    PowerMock.mockStatic(ExternalCompactionUtil.class);
-    List<RunningCompaction> runningCompactions = new ArrayList<>();
-    expect(ExternalCompactionUtil.getCompactionsRunningOnCompactors(context))
-        .andReturn(runningCompactions);
-    expect(ExternalCompactionUtil.getCompactorAddrs(context)).andReturn(Map.of()).anyTimes();
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-    TServerInstance instance = PowerMock.createNiceMock(TServerInstance.class);
-    expect(tservers.getCurrentServers()).andReturn(Collections.singleton(instance)).once();
-    HostAndPort tserverAddress = HostAndPort.fromString("localhost:9997");
-    expect(instance.getHostAndPort()).andReturn(tserverAddress).anyTimes();
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    TServerInstance tsi = PowerMock.createNiceMock(TServerInstance.class);
-    expect(tsi.getHostPort()).andReturn("localhost:9997").anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-    TCompactionQueueSummary queueSummary = PowerMock.createNiceMock(TCompactionQueueSummary.class);
-    expect(tsc.getCompactionQueueInfo(anyObject(), anyObject()))
-        .andReturn(Collections.singletonList(queueSummary)).anyTimes();
-    expect(queueSummary.getQueue()).andReturn("R2DQ").anyTimes();
-    expect(queueSummary.getPriority()).andReturn((short) 1).anyTimes();
-
-    ExternalCompactionId eci = ExternalCompactionId.generate(UUID.randomUUID());
-    TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class);
-    expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes();
-    TInfo trace = TraceUtil.traceInfo();
-    expect(tsc.reserveCompactionJob(trace, creds, "R2DQ", 1, "localhost:10241", eci.toString()))
-        .andReturn(job).anyTimes();
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-    expect(security.canPerformSystemActions(creds)).andReturn(true);
-
-    PowerMock.replayAll();
-
-    var coordinator = new TestCoordinator(finalizer, tservers, client, tsc, context, security);
-    coordinator.resetInternals();
-    assertEquals(0, coordinator.getQueues().size());
-    assertEquals(0, coordinator.getIndex().size());
-    assertEquals(0, coordinator.getRunning().size());
-    // Use coordinator.run() to populate the internal data structures. This is tested in a different
-    // test.
-    coordinator.run();
-
-    assertEquals(1, coordinator.getQueues().size());
-    QueueAndPriority qp = QueueAndPriority.get("R2DQ".intern(), (short) 1);
-    Map<Short,TreeSet<TServerInstance>> m = coordinator.getQueues().get("R2DQ".intern());
-    assertNotNull(m);
-    assertEquals(1, m.size());
-    assertTrue(m.containsKey((short) 1));
-    Set<TServerInstance> t = m.get((short) 1);
-    assertNotNull(t);
-    assertEquals(1, t.size());
-    TServerInstance queuedTsi = t.iterator().next();
-    assertEquals(tsi.getHostPortSession(), queuedTsi.getHostPortSession());
-    assertEquals(1, coordinator.getIndex().size());
-    assertTrue(coordinator.getIndex().containsKey(queuedTsi));
-    Set<QueueAndPriority> i = coordinator.getIndex().get(queuedTsi);
-    assertEquals(1, i.size());
-    assertEquals(qp, i.iterator().next());
-    assertEquals(0, coordinator.getRunning().size());
-
-    // Get the next job
-    TExternalCompactionJob createdJob =
-        coordinator.getCompactionJob(trace, creds, "R2DQ", "localhost:10241", eci.toString());
-    assertEquals(eci.toString(), createdJob.getExternalCompactionId());
-
-    assertEquals(1, coordinator.getQueues().size());
-    assertEquals(1, coordinator.getIndex().size());
-    assertEquals(1, coordinator.getRunning().size());
-    Entry<ExternalCompactionId,RunningCompaction> entry =
-        coordinator.getRunning().entrySet().iterator().next();
-    assertEquals(eci.toString(), entry.getKey().toString());
-    assertEquals("localhost:10241", entry.getValue().getCompactorAddress());
-    assertEquals(eci.toString(), entry.getValue().getJob().getExternalCompactionId());
-
-    PowerMock.verifyAll();
-    coordinator.resetInternals();
-    coordinator.close();
-
-  }
-
-  @Test
-  public void testGetCompactionJobNoJobs() throws Exception {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-
-    TCredentials creds = PowerMock.createNiceMock(TCredentials.class);
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-    expect(security.canPerformSystemActions(creds)).andReturn(true);
-
-    PowerMock.replayAll();
-
-    var coordinator = new TestCoordinator(finalizer, tservers, client, tsc, context, security);
-    coordinator.resetInternals();
-    TExternalCompactionJob job = coordinator.getCompactionJob(TraceUtil.traceInfo(), creds, "R2DQ",
-        "localhost:10240", UUID.randomUUID().toString());
-    assertNull(job.getExternalCompactionId());
-
-    PowerMock.verifyAll();
-    coordinator.resetInternals();
-    coordinator.close();
-  }
-
-  @Test
-  public void testCleanUpRunning() throws Exception {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
-
-    TCredentials creds = PowerMock.createNiceMock(TCredentials.class);
-
-    CompactionFinalizer finalizer = PowerMock.createNiceMock(CompactionFinalizer.class);
-    LiveTServerSet tservers = PowerMock.createNiceMock(LiveTServerSet.class);
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address).anyTimes();
-
-    TabletClientService.Client tsc = PowerMock.createNiceMock(TabletClientService.Client.class);
-
-    AuditedSecurityOperation security = PowerMock.createNiceMock(AuditedSecurityOperation.class);
-    expect(security.canPerformSystemActions(creds)).andReturn(true);
-
-    PowerMock.replayAll();
-
-    try (var coordinator =
-        new TestCoordinator(finalizer, tservers, client, tsc, context, security)) {
-      coordinator.resetInternals();
-
-      var ecid1 = ExternalCompactionId.generate(UUID.randomUUID());
-      var ecid2 = ExternalCompactionId.generate(UUID.randomUUID());
-      var ecid3 = ExternalCompactionId.generate(UUID.randomUUID());
-
-      coordinator.getRunning().put(ecid1, new RunningCompaction(new TExternalCompaction()));
-      coordinator.getRunning().put(ecid2, new RunningCompaction(new TExternalCompaction()));
-      coordinator.getRunning().put(ecid3, new RunningCompaction(new TExternalCompaction()));
-
-      coordinator.cleanUpRunning();
-
-      assertEquals(Set.of(ecid1, ecid2, ecid3), coordinator.getRunning().keySet());
-
-      coordinator.setMetadataCompactionIds(Set.of(ecid1, ecid2));
-
-      coordinator.cleanUpRunning();
-
-      assertEquals(Set.of(ecid1, ecid2), coordinator.getRunning().keySet());
-    }
-  }
-}
diff --git a/server/compactor/pom.xml b/server/compactor/pom.xml
index e7e4263..3696cab 100644
--- a/server/compactor/pom.xml
+++ b/server/compactor/pom.xml
@@ -24,17 +24,13 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>2.1.3-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-compactor</artifactId>
   <name>Apache Accumulo Compactor</name>
   <dependencies>
     <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.google.auto.service</groupId>
       <artifactId>auto-service</artifactId>
       <optional>true</optional>
@@ -79,30 +75,18 @@
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-slf4j2-impl</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.easymock</groupId>
-      <artifactId>easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.junit.vintage</groupId>
-      <artifactId>junit-vintage-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.powermock</groupId>
-      <artifactId>powermock-api-easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.powermock</groupId>
-      <artifactId>powermock-module-junit4</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
+  <build>
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-failsafe-plugin</artifactId>
+          <configuration>
+            <skipITs>true</skipITs>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
 </project>
diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
index 04e33a9..ebd195a 100644
--- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
+++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
@@ -18,20 +18,19 @@
  */
 package org.apache.accumulo.compactor;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MINUTES;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
 import java.net.UnknownHostException;
-import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -40,10 +39,12 @@
 import java.util.function.Supplier;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
@@ -60,14 +61,18 @@
 import org.apache.accumulo.core.data.NamespaceId;
 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.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
@@ -82,21 +87,17 @@
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.GarbageCollectionLogger;
-import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.compaction.CompactionInfo;
 import org.apache.accumulo.server.compaction.CompactionWatcher;
 import org.apache.accumulo.server.compaction.FileCompactor;
+import org.apache.accumulo.server.compaction.PausedCompactionMetrics;
 import org.apache.accumulo.server.compaction.RetryableThriftCall;
 import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
 import org.apache.accumulo.server.conf.TableConfiguration;
@@ -104,6 +105,7 @@
 import org.apache.accumulo.server.rpc.ServerAddress;
 import org.apache.accumulo.server.rpc.TServerUtils;
 import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.hadoop.fs.Path;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
@@ -111,34 +113,21 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 import io.micrometer.core.instrument.LongTaskTimer;
 import io.micrometer.core.instrument.MeterRegistry;
 
 public class Compactor extends AbstractServer implements MetricsProducer, CompactorService.Iface {
 
-  private static final SecureRandom random = new SecureRandom();
-
-  public static class CompactorServerOpts extends ServerOpts {
-    @Parameter(required = true, names = {"-q", "--queue"}, description = "compaction queue name")
-    private String queueName = null;
-
-    public String getQueueName() {
-      return queueName;
-    }
-  }
-
   private static final Logger LOG = LoggerFactory.getLogger(Compactor.class);
-  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
   private static final long TIME_BETWEEN_CANCEL_CHECKS = MINUTES.toMillis(5);
 
   private static final long TEN_MEGABYTES = 10485760;
 
   protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder();
 
-  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
   private final UUID compactorId = UUID.randomUUID();
   private final String queueName;
   protected final AtomicReference<ExternalCompactionId> currentCompactionId =
@@ -146,31 +135,26 @@
 
   private ServiceLock compactorLock;
   private ServerAddress compactorAddress = null;
+  private final PausedCompactionMetrics pausedMetrics = new PausedCompactionMetrics();
 
   // Exposed for tests
   protected volatile boolean shutdown = false;
 
   private final AtomicBoolean compactionRunning = new AtomicBoolean(false);
 
-  protected Compactor(CompactorServerOpts opts, String[] args) {
+  protected Compactor(ConfigOpts opts, String[] args) {
     super("compactor", opts, args);
-    queueName = opts.getQueueName();
+    queueName = super.getConfiguration().get(Property.COMPACTOR_QUEUE_NAME);
   }
 
   @Override
   public void registerMetrics(MeterRegistry registry) {
+    super.registerMetrics(registry);
     LongTaskTimer timer = LongTaskTimer.builder(METRICS_COMPACTOR_MAJC_STUCK)
         .description("Number and duration of stuck major compactions").register(registry);
     CompactionWatcher.setTimer(timer);
   }
 
-  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
-    ScheduledFuture<?> future =
-        schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
-            TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
-    ThreadPools.watchNonCriticalScheduledTask(future);
-  }
-
   protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor,
       long timeBetweenChecks) {
     ThreadPools.watchCriticalScheduledTask(schedExecutor.scheduleWithFixedDelay(
@@ -254,7 +238,7 @@
       public void lostLock(final LockLossReason reason) {
         Halt.halt(1, () -> {
           LOG.error("Compactor lost lock (reason = {}), exiting.", reason);
-          gcLogger.logGCInfo(getConfiguration());
+          getContext().getLowMemoryDetector().logGCInfo(getConfiguration());
         });
       }
 
@@ -265,12 +249,16 @@
     };
 
     try {
-      byte[] lockContent =
-          new ServerServices(hostPort, Service.COMPACTOR_CLIENT).toString().getBytes(UTF_8);
       for (int i = 0; i < 25; i++) {
         zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP);
 
-        if (compactorLock.tryLock(lw, lockContent)) {
+        ServiceDescriptors descriptors = new ServiceDescriptors();
+        for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT,
+            ThriftService.COMPACTOR}) {
+          descriptors.addService(new ServiceDescriptor(compactorId, svc, hostPort, this.queueName));
+        }
+
+        if (compactorLock.tryLock(lw, new ServiceLockData(descriptors))) {
           LOG.debug("Obtained Compactor lock {}", compactorLock.getLockPath());
           return;
         }
@@ -293,7 +281,10 @@
    * @throws UnknownHostException host unknown
    */
   protected ServerAddress startCompactorClientService() throws UnknownHostException {
-    var processor = ThriftProcessorTypes.getCompactorTProcessor(this, getContext());
+
+    ClientServiceHandler clientHandler =
+        new ClientServiceHandler(getContext(), new TransactionWatcher(getContext()));
+    var processor = ThriftProcessorTypes.getCompactorTProcessor(clientHandler, this, getContext());
     Property maxMessageSizeProperty =
         (getConfiguration().get(Property.COMPACTOR_MAX_MESSAGE_SIZE) != null
             ? Property.COMPACTOR_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
@@ -497,7 +488,8 @@
           aConfig = tConfig;
         }
 
-        final TabletFile outputFile = new TabletFile(new Path(job.getOutputFile()));
+        final ReferencedTabletFile outputFile =
+            new ReferencedTabletFile(new Path(job.getOutputFile()));
 
         final Map<StoredTabletFile,DataFileValue> files = new TreeMap<>();
         job.getFiles().forEach(f -> {
@@ -512,8 +504,9 @@
             .forEach(tis -> iters.add(SystemIteratorUtil.toIteratorSetting(tis)));
 
         ExtCEnv cenv = new ExtCEnv(JOB_HOLDER, queueName);
-        FileCompactor compactor = new FileCompactor(getContext(), extent, files, outputFile,
-            job.isPropagateDeletes(), cenv, iters, aConfig, tConfig.getCryptoService());
+        FileCompactor compactor =
+            new FileCompactor(getContext(), extent, files, outputFile, job.isPropagateDeletes(),
+                cenv, iters, aConfig, tConfig.getCryptoService(), pausedMetrics);
 
         LOG.trace("Starting compactor");
         started.countDown();
@@ -571,7 +564,7 @@
         sleepTime);
     // Add some random jitter to the sleep time, that averages out to sleep time. This will spread
     // compactors out evenly over time.
-    sleepTime = (long) (.9 * sleepTime + sleepTime * .2 * random.nextDouble());
+    sleepTime = (long) (.9 * sleepTime + sleepTime * .2 * RANDOM.get().nextDouble());
     LOG.trace("Sleeping {}ms based on {} compactors", sleepTime, numCompactors);
     return sleepTime;
   }
@@ -595,13 +588,12 @@
     MetricsInfo metricsInfo = getContext().getMetricsInfo();
     metricsInfo.addServiceTags(getApplicationName(), clientAddress);
 
-    metricsInfo.addMetricsProducers(this);
+    metricsInfo.addMetricsProducers(this, pausedMetrics);
     metricsInfo.init();
 
     var watcher = new CompactionWatcher(getConfiguration());
     var schedExecutor = ThreadPools.getServerThreadPools()
         .createGeneralScheduledExecutorService(getConfiguration());
-    startGCLogger(schedExecutor);
     startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
 
     LOG.info("Compactor started, waiting for work");
@@ -663,9 +655,9 @@
                       Float.toString((info.getEntriesRead() / (float) inputEntries) * 100);
                 }
                 String message = String.format(
-                    "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries",
+                    "Compaction in progress, read %d of %d input entries ( %s %s ), written %d entries, paused %d times",
                     info.getEntriesRead(), inputEntries, percentComplete, "%",
-                    info.getEntriesWritten());
+                    info.getEntriesWritten(), info.getTimesPaused());
                 watcher.run();
                 try {
                   LOG.debug("Updating coordinator with compaction progress: {}.", message);
@@ -780,7 +772,7 @@
         LOG.warn("Failed to close filesystem : {}", e.getMessage(), e);
       }
 
-      gcLogger.logGCInfo(getConfiguration());
+      getContext().getLowMemoryDetector().logGCInfo(getConfiguration());
       LOG.info("stop requested. exiting ... ");
       try {
         if (null != compactorLock) {
@@ -794,7 +786,7 @@
   }
 
   public static void main(String[] args) throws Exception {
-    try (Compactor compactor = new Compactor(new CompactorServerOpts(), args)) {
+    try (Compactor compactor = new Compactor(new ConfigOpts(), args)) {
       compactor.runServer();
     }
   }
diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java
index e68690b..c030230 100644
--- a/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java
+++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/ExtCEnv.java
@@ -102,12 +102,11 @@
     switch (job.getKind()) {
       case USER:
         return TCompactionReason.USER;
-      case CHOP:
-        return TCompactionReason.CHOP;
       case SELECTOR:
       case SYSTEM:
-      default:
         return TCompactionReason.SYSTEM;
+      default:
+        throw new IllegalStateException("Unknown compaction kind " + job.getKind());
     }
   }
 
diff --git a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java b/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java
deleted file mode 100644
index c747d93..0000000
--- a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java
+++ /dev/null
@@ -1,480 +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.compactor;
-
-import static org.easymock.EasyMock.expect;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.net.UnknownHostException;
-import java.util.List;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.atomic.LongAdder;
-import java.util.function.Supplier;
-
-import org.apache.accumulo.core.compaction.thrift.TCompactionState;
-import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate;
-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.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
-import org.apache.accumulo.core.metrics.MetricsInfo;
-import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
-import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
-import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.rpc.ServerAddress;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.api.easymock.PowerMock;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor;
-import org.powermock.modules.junit4.PowerMockRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({Compactor.class})
-@SuppressStaticInitializationFor({"org.apache.log4j.LogManager"})
-@PowerMockIgnore({"org.slf4j.*", "org.apache.logging.*", "org.apache.log4j.*",
-    "org.apache.commons.logging.*", "org.xml.*", "javax.xml.*", "org.w3c.dom.*",
-    "com.sun.org.apache.xerces.*"})
-public class CompactorTest {
-
-  public class SuccessfulCompaction implements Runnable {
-
-    protected final Logger LOG = LoggerFactory.getLogger(this.getClass());
-
-    protected final LongAdder totalInputEntries;
-    protected final LongAdder totalInputBytes;
-    protected final CountDownLatch started;
-    protected final CountDownLatch stopped;
-    protected final AtomicReference<Throwable> err;
-
-    public SuccessfulCompaction(LongAdder totalInputEntries, LongAdder totalInputBytes,
-        CountDownLatch started, CountDownLatch stopped, AtomicReference<Throwable> err) {
-      this.totalInputEntries = totalInputEntries;
-      this.totalInputBytes = totalInputBytes;
-      this.err = err;
-      this.started = started;
-      this.stopped = stopped;
-    }
-
-    @Override
-    public void run() {
-      try {
-        started.countDown();
-        UtilWaitThread.sleep(1000);
-      } catch (Exception e) {
-        err.set(e);
-      } finally {
-        stopped.countDown();
-      }
-    }
-  }
-
-  public class FailedCompaction extends SuccessfulCompaction {
-
-    public FailedCompaction(LongAdder totalInputEntries, LongAdder totalInputBytes,
-        CountDownLatch started, CountDownLatch stopped, AtomicReference<Throwable> err) {
-      super(totalInputEntries, totalInputBytes, started, stopped, err);
-    }
-
-    @Override
-    public void run() {
-      try {
-        started.countDown();
-        UtilWaitThread.sleep(1000);
-        throw new RuntimeException();
-      } catch (Exception e) {
-        err.set(e);
-      } finally {
-        stopped.countDown();
-      }
-    }
-  }
-
-  public class InterruptedCompaction extends SuccessfulCompaction {
-
-    public InterruptedCompaction(LongAdder totalInputEntries, LongAdder totalInputBytes,
-        CountDownLatch started, CountDownLatch stopped, AtomicReference<Throwable> err) {
-      super(totalInputEntries, totalInputBytes, started, stopped, err);
-    }
-
-    @Override
-    public void run() {
-      try {
-        started.countDown();
-        final Thread thread = Thread.currentThread();
-        Timer t = new Timer();
-        TimerTask task = new TimerTask() {
-          @Override
-          public void run() {
-            thread.interrupt();
-          }
-        };
-        t.schedule(task, 250);
-        Thread.sleep(1000);
-      } catch (Exception e) {
-        LOG.error("Compaction failed: {}", e.getMessage());
-        err.set(e);
-        throw new RuntimeException("Compaction failed", e);
-      } finally {
-        stopped.countDown();
-      }
-    }
-
-  }
-
-  public class SuccessfulCompactor extends Compactor {
-
-    private final Logger LOG = LoggerFactory.getLogger(SuccessfulCompactor.class);
-
-    private final Supplier<UUID> uuid;
-    private final ServerAddress address;
-    private final TExternalCompactionJob job;
-    private final ServerContext context;
-    private final ExternalCompactionId eci;
-    private volatile boolean completedCalled = false;
-    private volatile boolean failedCalled = false;
-    private TCompactionStatusUpdate latestState = null;
-
-    SuccessfulCompactor(Supplier<UUID> uuid, ServerAddress address, TExternalCompactionJob job,
-        ServerContext context, ExternalCompactionId eci) {
-      super(new CompactorServerOpts(), new String[] {"-q", "testQ"});
-      this.uuid = uuid;
-      this.address = address;
-      this.job = job;
-      this.context = context;
-      this.eci = eci;
-    }
-
-    @Override
-    protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {}
-
-    @Override
-    public ServerContext getContext() {
-      return this.context;
-    }
-
-    @Override
-    protected void announceExistence(HostAndPort clientAddress)
-        throws KeeperException, InterruptedException {}
-
-    @Override
-    protected ServerAddress startCompactorClientService() throws UnknownHostException {
-      return this.address;
-    }
-
-    @Override
-    protected TExternalCompactionJob getNextJob(Supplier<UUID> uuid)
-        throws RetriesExceededException {
-      LOG.info("Attempting to get next job, eci = {}", eci);
-      currentCompactionId.set(eci);
-      this.shutdown = true;
-      return job;
-    }
-
-    @Override
-    protected synchronized void checkIfCanceled() {}
-
-    @Override
-    protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries,
-        LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped,
-        AtomicReference<Throwable> err) {
-      return new SuccessfulCompaction(totalInputEntries, totalInputBytes, started, stopped, err);
-    }
-
-    @Override
-    protected Supplier<UUID> getNextId() {
-      return uuid;
-    }
-
-    @Override
-    protected void updateCompactionState(TExternalCompactionJob job, TCompactionStatusUpdate update)
-        throws RetriesExceededException {
-      latestState = update;
-    }
-
-    @Override
-    protected void updateCompactionFailed(TExternalCompactionJob job)
-        throws RetriesExceededException {
-      failedCalled = true;
-    }
-
-    @Override
-    protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats)
-        throws RetriesExceededException {
-      completedCalled = true;
-    }
-
-    public TCompactionState getLatestState() {
-      return latestState.getState();
-    }
-
-    public boolean isCompletedCalled() {
-      return completedCalled;
-    }
-
-    public boolean isFailedCalled() {
-      return failedCalled;
-    }
-
-  }
-
-  public class FailedCompactor extends SuccessfulCompactor {
-
-    FailedCompactor(Supplier<UUID> uuid, ServerAddress address, TExternalCompactionJob job,
-        ServerContext context, ExternalCompactionId eci) {
-      super(uuid, address, job, context, eci);
-    }
-
-    @Override
-    protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries,
-        LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped,
-        AtomicReference<Throwable> err) {
-      return new FailedCompaction(totalInputEntries, totalInputBytes, started, stopped, err);
-    }
-  }
-
-  public class InterruptedCompactor extends SuccessfulCompactor {
-
-    InterruptedCompactor(Supplier<UUID> uuid, ServerAddress address, TExternalCompactionJob job,
-        ServerContext context, ExternalCompactionId eci) {
-      super(uuid, address, job, context, eci);
-    }
-
-    @Override
-    protected Runnable createCompactionJob(TExternalCompactionJob job, LongAdder totalInputEntries,
-        LongAdder totalInputBytes, CountDownLatch started, CountDownLatch stopped,
-        AtomicReference<Throwable> err) {
-      return new InterruptedCompaction(totalInputEntries, totalInputBytes, started, stopped, err);
-    }
-
-  }
-
-  @Test
-  public void testCheckTime() throws Exception {
-    assertEquals(1, Compactor.calculateProgressCheckTime(1024));
-    assertEquals(1, Compactor.calculateProgressCheckTime(1048576));
-    assertEquals(1, Compactor.calculateProgressCheckTime(10485760));
-    assertEquals(10, Compactor.calculateProgressCheckTime(104857600));
-    assertEquals(102, Compactor.calculateProgressCheckTime(1024 * 1024 * 1024));
-  }
-
-  @Test
-  public void testCompactionSucceeds() throws Exception {
-    UUID uuid = UUID.randomUUID();
-    Supplier<UUID> supplier = () -> uuid;
-
-    ExternalCompactionId eci = ExternalCompactionId.generate(supplier.get());
-
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.methods(Halt.class, "halt"));
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address);
-
-    TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class);
-    TKeyExtent extent = PowerMock.createNiceMock(TKeyExtent.class);
-    expect(job.isSetExternalCompactionId()).andReturn(true).anyTimes();
-    expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes();
-    expect(job.getExtent()).andReturn(extent).anyTimes();
-    expect(extent.getTable()).andReturn("testTable".getBytes()).anyTimes();
-
-    var conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    conf.set(Property.INSTANCE_ZK_TIMEOUT, "1d");
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(conf).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    ZooReaderWriter zrw = PowerMock.createNiceMock(ZooReaderWriter.class);
-    ZooKeeper zk = PowerMock.createNiceMock(ZooKeeper.class);
-    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
-    expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
-    VolumeManagerImpl vm = PowerMock.createNiceMock(VolumeManagerImpl.class);
-    expect(context.getVolumeManager()).andReturn(vm);
-    vm.close();
-
-    PowerMock.replayAll();
-
-    SuccessfulCompactor c = new SuccessfulCompactor(supplier, client, job, context, eci);
-    c.run();
-
-    PowerMock.verifyAll();
-    c.close();
-
-    assertTrue(c.isCompletedCalled());
-    assertFalse(c.isFailedCalled());
-  }
-
-  @Test
-  public void testCompactionFails() throws Exception {
-    UUID uuid = UUID.randomUUID();
-    Supplier<UUID> supplier = () -> uuid;
-
-    ExternalCompactionId eci = ExternalCompactionId.generate(supplier.get());
-
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.methods(Halt.class, "halt"));
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address);
-
-    TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class);
-    TKeyExtent extent = new KeyExtent(TableId.of("testTable"), null, null).toThrift();
-
-    expect(job.isSetExternalCompactionId()).andReturn(true).anyTimes();
-    expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes();
-    expect(job.getExtent()).andReturn(extent).anyTimes();
-
-    var conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    conf.set(Property.INSTANCE_ZK_TIMEOUT, "1d");
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(conf).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    ZooReaderWriter zrw = PowerMock.createNiceMock(ZooReaderWriter.class);
-    ZooKeeper zk = PowerMock.createNiceMock(ZooKeeper.class);
-    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
-    expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
-    VolumeManagerImpl vm = PowerMock.createNiceMock(VolumeManagerImpl.class);
-    expect(context.getVolumeManager()).andReturn(vm);
-    vm.close();
-
-    PowerMock.replayAll();
-
-    FailedCompactor c = new FailedCompactor(supplier, client, job, context, eci);
-    c.run();
-
-    PowerMock.verifyAll();
-    c.close();
-
-    assertFalse(c.isCompletedCalled());
-    assertTrue(c.isFailedCalled());
-    assertEquals(TCompactionState.FAILED, c.getLatestState());
-  }
-
-  @Test
-  public void testCompactionInterrupted() throws Exception {
-    UUID uuid = UUID.randomUUID();
-    Supplier<UUID> supplier = () -> uuid;
-
-    ExternalCompactionId eci = ExternalCompactionId.generate(supplier.get());
-
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.methods(Halt.class, "halt"));
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-
-    ServerAddress client = PowerMock.createNiceMock(ServerAddress.class);
-    HostAndPort address = HostAndPort.fromString("localhost:10240");
-    expect(client.getAddress()).andReturn(address);
-
-    TExternalCompactionJob job = PowerMock.createNiceMock(TExternalCompactionJob.class);
-    TKeyExtent extent = PowerMock.createNiceMock(TKeyExtent.class);
-    expect(job.isSetExternalCompactionId()).andReturn(true).anyTimes();
-    expect(job.getExternalCompactionId()).andReturn(eci.toString()).anyTimes();
-    expect(job.getExtent()).andReturn(extent).anyTimes();
-    expect(extent.getTable()).andReturn("testTable".getBytes()).anyTimes();
-
-    var conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    conf.set(Property.INSTANCE_ZK_TIMEOUT, "1d");
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(conf).anyTimes();
-    MetricsInfo metricsInfo = PowerMock.createNiceMock(MetricsInfo.class);
-    expect(context.getMetricsInfo()).andReturn(metricsInfo).anyTimes();
-
-    ZooReaderWriter zrw = PowerMock.createNiceMock(ZooReaderWriter.class);
-    ZooKeeper zk = PowerMock.createNiceMock(ZooKeeper.class);
-    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
-    expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
-    VolumeManagerImpl vm = PowerMock.createNiceMock(VolumeManagerImpl.class);
-    expect(context.getVolumeManager()).andReturn(vm);
-    vm.close();
-
-    PowerMock.replayAll();
-
-    InterruptedCompactor c = new InterruptedCompactor(supplier, client, job, context, eci);
-    c.run();
-
-    PowerMock.verifyAll();
-    c.close();
-
-    assertFalse(c.isCompletedCalled());
-    assertTrue(c.isFailedCalled());
-    assertEquals(TCompactionState.CANCELLED, c.getLatestState());
-  }
-
-  @Test
-  public void testCompactionWaitProperty() {
-    PowerMock.resetAll();
-    PowerMock.suppress(PowerMock.methods(Halt.class, "halt"));
-    PowerMock.suppress(PowerMock.constructor(AbstractServer.class));
-
-    var conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    conf.set(Property.COMPACTOR_MAX_JOB_WAIT_TIME, "800ms");
-
-    ServerContext context = PowerMock.createNiceMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(conf).anyTimes();
-    expect(context.getZooKeeperRoot()).andReturn("test").anyTimes();
-    ZooCache zkc = PowerMock.createNiceMock(ZooCache.class);
-    expect(zkc.getChildren("test/compactors/testQ")).andReturn(List.of("compactor_1")).anyTimes();
-    expect(context.getZooCache()).andReturn(zkc).anyTimes();
-
-    PowerMock.replayAll();
-
-    try (var c = new SuccessfulCompactor(null, null, null, context, null)) {
-      Long maxWait = c.getWaitTimeBetweenCompactionChecks();
-      // compaction jitter means maxWait is between 0.9 and 1.1 of the desired value.
-      assertTrue(maxWait >= 720L);
-      assertTrue(maxWait <= 968L);
-    }
-
-    PowerMock.verifyAll();
-  }
-
-}
diff --git a/server/compactor/src/test/resources/log4j2-test.properties b/server/compactor/src/test/resources/log4j2-test.properties
deleted file mode 100644
index f3095a1..0000000
--- a/server/compactor/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,35 +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.
-#
-
-status = info
-dest = err
-name = AccumuloCoreTestLoggingProperties
-
-appender.console.type = Console
-appender.console.name = STDOUT
-appender.console.target = SYSTEM_OUT
-appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = [%t] %-5p %c %x - %m%n
-
-#logger.01.name = org.apache.accumulo.core.iterators.system.VisibilityFilter
-#logger.01.level = fatal
-
-rootLogger.level = info
-rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/server/gc/pom.xml b/server/gc/pom.xml
index 1c2d9b9..d39e2b5 100644
--- a/server/gc/pom.xml
+++ b/server/gc/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-gc</artifactId>
@@ -41,10 +41,6 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>io.micrometer</groupId>
       <artifactId>micrometer-core</artifactId>
     </dependency>
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java
index cd1753b..4cb6f3b 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.gc;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.DIR;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
@@ -41,9 +42,7 @@
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -54,7 +53,7 @@
 import org.apache.accumulo.core.gc.ReferenceDirectory;
 import org.apache.accumulo.core.gc.ReferenceFile;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-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.StoredTabletFile;
 import org.apache.accumulo.core.metadata.ValidationUtil;
@@ -65,15 +64,12 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.gc.GcVolumeUtil;
-import org.apache.accumulo.server.replication.proto.Replication;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.zookeeper.KeeperException;
@@ -82,9 +78,6 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Maps;
-import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A single garbage collection performed on a table (Root, MD) or all User tables.
@@ -182,9 +175,8 @@
     if (level == Ample.DataLevel.ROOT) {
       tabletStream = Stream.of(context.getAmple().readTablet(RootTable.EXTENT, DIR, FILES, SCANS));
     } else {
-      var tabletsMetadata = TabletsMetadata.builder(context).scanTable(level.metaTable())
-          .checkConsistency().fetch(DIR, FILES, SCANS).build();
-      tabletStream = tabletsMetadata.stream();
+      tabletStream = TabletsMetadata.builder(context).scanTable(level.metaTable())
+          .checkConsistency().fetch(DIR, FILES, SCANS).build().stream();
     }
 
     // there is a lot going on in this "one line" so see below for more info
@@ -203,13 +195,12 @@
       // combine all the entries read from file and scan columns in the metadata table
       Stream<StoredTabletFile> stfStream = tm.getFiles().stream();
       // map the files to Reference objects
-      var fileStream = stfStream.map(f -> ReferenceFile.forFile(tableId, f.getMetaUpdateDelete()));
+      var fileStream = stfStream.map(f -> ReferenceFile.forFile(tableId, f));
 
       // scans are normally empty, so only introduce a layer of indirection when needed
       final var tmScans = tm.getScans();
       if (!tmScans.isEmpty()) {
-        var scanStream =
-            tmScans.stream().map(s -> ReferenceFile.forScan(tableId, s.getMetaUpdateDelete()));
+        var scanStream = tmScans.stream().map(s -> ReferenceFile.forScan(tableId, s));
         fileStream = Stream.concat(fileStream, scanStream);
       }
       // if dirName is populated, then we have a tablet directory aka srv:dir
@@ -222,7 +213,7 @@
     });
 
     var scanServerRefs = context.getAmple().getScanServerFileReferences()
-        .map(sfr -> ReferenceFile.forScan(sfr.getTableId(), sfr.getPathStr()));
+        .map(sfr -> ReferenceFile.forScan(sfr.getTableId(), sfr));
 
     return Stream.concat(tabletReferences, scanServerRefs);
   }
@@ -262,7 +253,7 @@
         }
         ioe.addSuppressed(e);
         log.error("Error getting tables from ZooKeeper, retrying in {} seconds", retries, e);
-        UtilWaitThread.sleepUninterruptibly(retries, TimeUnit.SECONDS);
+        sleepUninterruptibly(retries, TimeUnit.SECONDS);
       }
     }
     throw ioe;
@@ -273,7 +264,7 @@
       throws TableNotFoundException {
     final VolumeManager fs = context.getVolumeManager();
     var metadataLocation = level == Ample.DataLevel.ROOT
-        ? context.getZooKeeperRoot() + " for " + RootTable.NAME : level.metaTable();
+        ? context.getZooKeeperRoot() + " for " + AccumuloTable.ROOT.tableName() : level.metaTable();
 
     if (inSafeMode()) {
       System.out.println("SAFEMODE: There are " + confirmedDeletes.size()
@@ -294,7 +285,7 @@
     ExecutorService deleteThreadPool = ThreadPools.getServerThreadPools()
         .createExecutorService(config, Property.GC_DELETE_THREADS);
 
-    final List<Pair<Path,Path>> replacements = context.getVolumeReplacements();
+    final Map<Path,Path> replacements = context.getVolumeReplacements();
 
     for (final GcCandidate delete : confirmedDeletes.values()) {
 
@@ -303,8 +294,8 @@
 
         try {
           Path fullPath;
-          Path switchedDelete =
-              VolumeUtil.switchVolume(delete.getPath(), VolumeManager.FileType.TABLE, replacements);
+          Path switchedDelete = VolumeUtil.switchVolume(new Path(delete.getPath()),
+              VolumeManager.FileType.TABLE, replacements);
           if (switchedDelete != null) {
             // actually replacing the volumes in the metadata table would be tricky because the
             // entries would be different rows. So it could not be
@@ -416,30 +407,6 @@
     inUse += i;
   }
 
-  @Override
-  @Deprecated
-  public Iterator<Map.Entry<String,Replication.Status>> getReplicationNeededIterator() {
-    AccumuloClient client = context;
-    try {
-      Scanner s = org.apache.accumulo.core.replication.ReplicationTable.getScanner(client);
-      org.apache.accumulo.core.replication.ReplicationSchema.StatusSection.limit(s);
-      return Iterators.transform(s.iterator(), input -> {
-        String file = input.getKey().getRow().toString();
-        Replication.Status stat;
-        try {
-          stat = Replication.Status.parseFrom(input.getValue().get());
-        } catch (InvalidProtocolBufferException e) {
-          log.warn("Could not deserialize protobuf for: {}", input.getKey());
-          stat = null;
-        }
-        return Maps.immutableEntry(file, stat);
-      });
-    } catch (org.apache.accumulo.core.replication.ReplicationTableOfflineException e) {
-      // No elements that we need to preclude
-      return Collections.emptyIterator();
-    }
-  }
-
   @VisibleForTesting
   static void minimizeDeletes(SortedMap<String,GcCandidate> confirmedDeletes,
       List<GcCandidate> processedDeletes, VolumeManager fs, Logger logger) {
@@ -505,16 +472,6 @@
   }
 
   /**
-   * Checks if InUse Candidates can be removed.
-   *
-   * @return value of {@link Property#GC_REMOVE_IN_USE_CANDIDATES}
-   */
-  @Override
-  public boolean canRemoveInUseCandidates() {
-    return context.getConfiguration().getBoolean(Property.GC_REMOVE_IN_USE_CANDIDATES);
-  }
-
-  /**
    * Moves a file to trash. If this garbage collector is not using trash, this method returns false
    * and leaves the file alone. If the file is missing, this method returns false as opposed to
    * throwing an exception.
@@ -524,10 +481,6 @@
    */
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = context.getVolumeManager();
-    if (!isUsingTrash()) {
-      log.trace("Accumulo Trash is disabled. Skipped for {}", path);
-      return false;
-    }
     try {
       boolean success = fs.moveToTrash(path);
       log.trace("Accumulo Trash enabled, moving to trash succeeded?: {}", success);
@@ -539,17 +492,6 @@
   }
 
   /**
-   * Checks if the volume manager should move files to the trash rather than delete them.
-   *
-   * @return true if trash is used
-   */
-  boolean isUsingTrash() {
-    @SuppressWarnings("removal")
-    Property p = Property.GC_TRASH_IGNORE;
-    return !config.getBoolean(p);
-  }
-
-  /**
    * Gets the batch size for garbage collecting.
    *
    * @return candidate batch size.
@@ -588,9 +530,9 @@
   @Override
   public Set<TableId> getCandidateTableIDs() throws InterruptedException {
     if (level == DataLevel.ROOT) {
-      return Set.of(RootTable.ID);
+      return Set.of(AccumuloTable.ROOT.tableId());
     } else if (level == DataLevel.METADATA) {
-      return Set.of(MetadataTable.ID);
+      return Set.of(AccumuloTable.METADATA.tableId());
     } else if (level == DataLevel.USER) {
       Set<TableId> tableIds = new HashSet<>();
       getTableIDs().forEach((k, v) -> {
@@ -600,8 +542,8 @@
           tableIds.add(k);
         }
       });
-      tableIds.remove(MetadataTable.ID);
-      tableIds.remove(RootTable.ID);
+      tableIds.remove(AccumuloTable.METADATA.tableId());
+      tableIds.remove(AccumuloTable.ROOT.tableId());
       return tableIds;
     } else {
       throw new IllegalArgumentException("Unexpected level in GC Env: " + this.level.name());
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index ecfaeab..d6f3c63 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -29,19 +29,13 @@
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
-import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.server.ServerContext;
@@ -53,7 +47,6 @@
 import org.apache.accumulo.server.manager.state.TabletStateStore;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,7 +61,6 @@
 
   private final ServerContext context;
   private final VolumeManager fs;
-  private final boolean useTrash;
   private final LiveTServerSet liveServers;
   private final WalStateManager walMarker;
   private final Iterable<TabletLocationState> store;
@@ -78,13 +70,11 @@
    *
    * @param context the collection server's context
    * @param fs volume manager to use
-   * @param useTrash true to move files to trash rather than delete them
    */
   GarbageCollectWriteAheadLogs(final ServerContext context, final VolumeManager fs,
-      final LiveTServerSet liveServers, boolean useTrash) {
+      final LiveTServerSet liveServers) {
     this.context = context;
     this.fs = fs;
-    this.useTrash = useTrash;
     this.liveServers = liveServers;
     this.walMarker = new WalStateManager(context);
     this.store = () -> Iterators.concat(
@@ -98,16 +88,14 @@
    *
    * @param context the collection server's context
    * @param fs volume manager to use
-   * @param useTrash true to move files to trash rather than delete them
    * @param liveTServerSet a started LiveTServerSet instance
    */
   @VisibleForTesting
-  GarbageCollectWriteAheadLogs(ServerContext context, VolumeManager fs, boolean useTrash,
+  GarbageCollectWriteAheadLogs(ServerContext context, VolumeManager fs,
       LiveTServerSet liveTServerSet, WalStateManager walMarker,
       Iterable<TabletLocationState> store) {
     this.context = context;
     this.fs = fs;
-    this.useTrash = useTrash;
     this.liveServers = liveTServerSet;
     this.walMarker = walMarker;
     this.store = store;
@@ -169,21 +157,6 @@
       log.info(String.format("%d log entries scanned in %.2f seconds", count,
           (logEntryScanStop - fileScanStop) / 1000.));
 
-      Span span3 = TraceUtil.startSpan(this.getClass(), "removeReplicationEntries");
-      try (Scope scope = span3.makeCurrent()) {
-        count = removeReplicationEntries(uuidToTServer);
-      } catch (Exception ex) {
-        log.error("Unable to scan replication table", ex);
-        TraceUtil.setException(span3, ex, false);
-        return;
-      } finally {
-        span3.end();
-      }
-
-      long replicationEntryScanStop = System.currentTimeMillis();
-      log.info(String.format("%d replication entries scanned in %.2f seconds", count,
-          (replicationEntryScanStop - logEntryScanStop) / 1000.));
-
       long removeStop;
       Span span4 = TraceUtil.startSpan(this.getClass(), "removeFiles");
       try (Scope scope = span4.makeCurrent()) {
@@ -253,7 +226,7 @@
 
   private long removeFile(Path path) {
     try {
-      if (!useTrash || !fs.moveToTrash(path)) {
+      if (!fs.moveToTrash(path)) {
         fs.deleteRecursively(path);
       }
       return 1;
@@ -284,10 +257,6 @@
     return count;
   }
 
-  private UUID path2uuid(Path path) {
-    return UUID.fromString(path.getName());
-  }
-
   private Map<UUID,TServerInstance> removeEntriesInUse(Map<TServerInstance,Set<UUID>> candidates,
       Set<TServerInstance> liveServers, Map<UUID,Pair<WalState,Path>> logsState,
       Map<UUID,Path> recoveryLogs) {
@@ -314,16 +283,14 @@
       }
       // Tablet is being recovered and has WAL references, remove all the WALs for the dead server
       // that made the WALs.
-      for (Collection<String> wals : state.walogs) {
-        for (String wal : wals) {
-          UUID walUUID = path2uuid(new Path(wal));
-          TServerInstance dead = result.get(walUUID);
-          // There's a reference to a log file, so skip that server's logs
-          Set<UUID> idsToIgnore = candidates.remove(dead);
-          if (idsToIgnore != null) {
-            result.keySet().removeAll(idsToIgnore);
-            recoveryLogs.keySet().removeAll(idsToIgnore);
-          }
+      for (LogEntry wal : state.walogs) {
+        UUID walUUID = wal.getUniqueID();
+        TServerInstance dead = result.get(walUUID);
+        // There's a reference to a log file, so skip that server's logs
+        Set<UUID> idsToIgnore = candidates.remove(dead);
+        if (idsToIgnore != null) {
+          result.keySet().removeAll(idsToIgnore);
+          recoveryLogs.keySet().removeAll(idsToIgnore);
         }
       }
     }
@@ -346,39 +313,6 @@
     return result;
   }
 
-  @Deprecated
-  protected int removeReplicationEntries(Map<UUID,TServerInstance> candidates) {
-    try {
-      try {
-        final Scanner s = org.apache.accumulo.core.replication.ReplicationTable.getScanner(context);
-        org.apache.accumulo.core.replication.ReplicationSchema.StatusSection.limit(s);
-        for (Entry<Key,Value> entry : s) {
-          UUID id = path2uuid(new Path(entry.getKey().getRow().toString()));
-          candidates.remove(id);
-          log.info("Ignore closed log " + id + " because it is being replicated");
-        }
-      } catch (org.apache.accumulo.core.replication.ReplicationTableOfflineException ex) {
-        return candidates.size();
-      }
-
-      final Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      scanner.fetchColumnFamily(ReplicationSection.COLF);
-      scanner.setRange(ReplicationSection.getRange());
-      for (Entry<Key,Value> entry : scanner) {
-        Text file = new Text();
-        ReplicationSection.getFile(entry.getKey(), file);
-        UUID id = path2uuid(new Path(file.toString()));
-        candidates.remove(id);
-        log.info("Ignore closed log " + id + " because it is being replicated");
-      }
-
-      return candidates.size();
-    } catch (TableNotFoundException e) {
-      log.error("Failed to scan metadata table", e);
-      throw new IllegalArgumentException(e);
-    }
-  }
-
   /**
    * Scans log markers. The map passed in is populated with the log ids.
    *
@@ -416,7 +350,7 @@
       if (fs.exists(recoveryDir)) {
         for (FileStatus status : fs.listStatus(recoveryDir)) {
           try {
-            UUID logId = path2uuid(status.getPath());
+            UUID logId = UUID.fromString(status.getPath().getName());
             result.put(logId, status.getPath());
           } catch (IllegalArgumentException iae) {
             log.debug("Ignoring file " + status.getPath() + " because it doesn't look like a uuid");
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java
index 6800b9a..eb6b8fe 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionAlgorithm.java
@@ -29,7 +29,6 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -44,13 +43,10 @@
 import org.apache.accumulo.core.metadata.schema.Ample.GcCandidateType;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.PeekingIterator;
 
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
@@ -147,62 +143,62 @@
     List<GcCandidate> candidateEntriesToBeDeleted = new ArrayList<>();
     Set<TableId> tableIdsBefore = gce.getCandidateTableIDs();
     Set<TableId> tableIdsSeen = new HashSet<>();
-    Iterator<Reference> iter = gce.getReferences().iterator();
-    while (iter.hasNext()) {
-      Reference ref = iter.next();
-      tableIdsSeen.add(ref.getTableId());
+    try (Stream<Reference> references = gce.getReferences()) {
+      references.forEach(ref -> {
+        tableIdsSeen.add(ref.getTableId());
 
-      if (ref.isDirectory()) {
-        var dirReference = (ReferenceDirectory) ref;
-        ServerColumnFamily.validateDirCol(dirReference.getTabletDir());
+        if (ref.isDirectory()) {
+          var dirReference = (ReferenceDirectory) ref;
+          ServerColumnFamily.validateDirCol(dirReference.getTabletDir());
 
-        String dir = "/" + dirReference.tableId + "/" + dirReference.getTabletDir();
+          String dir = "/" + dirReference.tableId + "/" + dirReference.getTabletDir();
 
-        dir = makeRelative(dir, 2);
+          dir = makeRelative(dir, 2);
 
-        GcCandidate gcTemp = candidateMap.remove(dir);
-        if (gcTemp != null) {
-          log.debug("Directory Candidate was still in use by dir ref: {}", dir);
-          // Do not add dir candidates to candidateEntriesToBeDeleted as they are only created once.
-        }
-      } else {
-        String reference = ref.getMetadataEntry();
-        if (reference.startsWith("/")) {
-          log.debug("Candidate {} has a relative path, prepend tableId {}", reference,
-              ref.getTableId());
-          reference = "/" + ref.getTableId() + ref.getMetadataEntry();
-        } else if (!reference.contains(":") && !reference.startsWith("../")) {
-          throw new RuntimeException("Bad file reference " + reference);
-        }
+          GcCandidate gcTemp = candidateMap.remove(dir);
+          if (gcTemp != null) {
+            log.debug("Directory Candidate was still in use by dir ref: {}", dir);
+            // Do not add dir candidates to candidateEntriesToBeDeleted as they are only created
+            // once.
+          }
+        } else {
+          String reference = ref.getMetadataPath();
+          if (reference.startsWith("/")) {
+            log.debug("Candidate {} has a relative path, prepend tableId {}", reference,
+                ref.getTableId());
+            reference = "/" + ref.getTableId() + ref.getMetadataPath();
+          } else if (!reference.contains(":") && !reference.startsWith("../")) {
+            throw new RuntimeException("Bad file reference " + reference);
+          }
 
-        String relativePath = makeRelative(reference, 3);
+          String relativePath = makeRelative(reference, 3);
 
-        // WARNING: This line is EXTREMELY IMPORTANT.
-        // You MUST REMOVE candidates that are still in use
-        GcCandidate gcTemp = candidateMap.remove(relativePath);
-        if (gcTemp != null) {
-          log.debug("File Candidate was still in use: {}", relativePath);
-          // Prevent deletion of candidates that are still in use by scans, because they won't be
-          // recreated once the scan is finished.
-          if (!ref.isScan()) {
-            candidateEntriesToBeDeleted.add(gcTemp);
+          // WARNING: This line is EXTREMELY IMPORTANT.
+          // You MUST REMOVE candidates that are still in use
+          GcCandidate gcTemp = candidateMap.remove(relativePath);
+          if (gcTemp != null) {
+            log.debug("File Candidate was still in use: {}", relativePath);
+            // Prevent deletion of candidates that are still in use by scans, because they won't be
+            // recreated once the scan is finished.
+            if (!ref.isScan()) {
+              candidateEntriesToBeDeleted.add(gcTemp);
+            }
+          }
+
+          String dir = relativePath.substring(0, relativePath.lastIndexOf('/'));
+          GcCandidate gcT = candidateMap.remove(dir);
+          if (gcT != null) {
+            log.debug("Directory Candidate was still in use by file ref: {}", relativePath);
+            // Do not add dir candidates to candidateEntriesToBeDeleted as they are only created
+            // once.
           }
         }
-
-        String dir = relativePath.substring(0, relativePath.lastIndexOf('/'));
-        GcCandidate gcT = candidateMap.remove(dir);
-        if (gcT != null) {
-          log.debug("Directory Candidate was still in use by file ref: {}", relativePath);
-          // Do not add dir candidates to candidateEntriesToBeDeleted as they are only created once.
-        }
-      }
+      });
     }
     Set<TableId> tableIdsAfter = gce.getCandidateTableIDs();
     ensureAllTablesChecked(Collections.unmodifiableSet(tableIdsBefore),
         Collections.unmodifiableSet(tableIdsSeen), Collections.unmodifiableSet(tableIdsAfter));
-    if (gce.canRemoveInUseCandidates()) {
-      gce.deleteGcCandidates(candidateEntriesToBeDeleted, GcCandidateType.INUSE);
-    }
+    gce.deleteGcCandidates(candidateEntriesToBeDeleted, GcCandidateType.INUSE);
   }
 
   private long removeBlipCandidates(GarbageCollectionEnvironment gce,
@@ -287,44 +283,6 @@
     }
   }
 
-  protected void confirmDeletesFromReplication(GarbageCollectionEnvironment gce,
-      SortedMap<String,GcCandidate> candidateMap) {
-    var replicationNeededIterator = gce.getReplicationNeededIterator();
-    var candidateMapIterator = candidateMap.entrySet().iterator();
-
-    PeekingIterator<Entry<String,Status>> pendingReplication =
-        Iterators.peekingIterator(replicationNeededIterator);
-    PeekingIterator<Entry<String,GcCandidate>> candidates =
-        Iterators.peekingIterator(candidateMapIterator);
-    while (pendingReplication.hasNext() && candidates.hasNext()) {
-      Entry<String,Status> pendingReplica = pendingReplication.peek();
-      Entry<String,GcCandidate> candidate = candidates.peek();
-
-      String filePendingReplication = pendingReplica.getKey();
-      String fullPathCandidate = candidate.getValue().getPath();
-
-      int comparison = filePendingReplication.compareTo(fullPathCandidate);
-      if (comparison < 0) {
-        pendingReplication.next();
-      } else if (comparison > 1) {
-        candidates.next();
-      } else {
-        // We want to advance both, and try to delete the candidate if we can
-        candidates.next();
-        pendingReplication.next();
-
-        // We cannot delete a file if it is still needed for replication
-        @SuppressWarnings("deprecation")
-        boolean safeToRemove = org.apache.accumulo.server.replication.StatusUtil
-            .isSafeForRemoval(pendingReplica.getValue());
-        if (!safeToRemove) {
-          // If it must be replicated, we must remove it from the candidate set to prevent deletion
-          candidates.remove();
-        }
-      }
-    }
-  }
-
   private void cleanUpDeletedTableDirs(GarbageCollectionEnvironment gce,
       SortedMap<String,GcCandidate> candidateMap) throws InterruptedException, IOException {
     HashSet<TableId> tableIdsWithDeletes = new HashSet<>();
@@ -358,7 +316,6 @@
     try (Scope scope = confirmDeletesSpan.makeCurrent()) {
       blips = removeBlipCandidates(gce, candidateMap);
       removeCandidatesInUse(gce, candidateMap);
-      confirmDeletesFromReplication(gce, candidateMap);
     } catch (Exception e) {
       TraceUtil.setException(confirmDeletesSpan, e, true);
       throw e;
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java
index 96aae86..3c373e9 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectionEnvironment.java
@@ -23,42 +23,31 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.gc.GcCandidate;
 import org.apache.accumulo.core.gc.Reference;
 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.schema.Ample.GcCandidateType;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 
 public interface GarbageCollectionEnvironment {
 
   /**
    * Return an iterator which points to a list of paths to files and dirs which are candidates for
-   * deletion from a given table, {@link RootTable#NAME} or {@link MetadataTable#NAME}
+   * deletion from a given table, {@link org.apache.accumulo.core.metadata.AccumuloTable#ROOT} or
+   * {@link org.apache.accumulo.core.metadata.AccumuloTable#METADATA}
    *
    * @return an iterator referencing a List containing deletion candidates
    */
   Iterator<GcCandidate> getCandidates() throws TableNotFoundException;
 
   /**
-   * Used for determining if deletion of InUse candidates is enabled.
-   *
-   * @return value of {@link Property#GC_REMOVE_IN_USE_CANDIDATES}
-   */
-  boolean canRemoveInUseCandidates();
-
-  /**
    * Given an iterator to a deletion candidate list, return a sub-list of candidates which fit
    * within provided memory constraints.
    *
@@ -69,7 +58,8 @@
 
   /**
    * Fetch a list of paths for all bulk loads in progress (blip) from a given table,
-   * {@link RootTable#NAME} or {@link MetadataTable#NAME}
+   * {@link org.apache.accumulo.core.metadata.AccumuloTable#ROOT} or
+   * {@link org.apache.accumulo.core.metadata.AccumuloTable#METADATA}
    *
    * @return The list of files for each bulk load currently in progress.
    */
@@ -113,7 +103,7 @@
       throws TableNotFoundException;
 
   /**
-   * Delete in-use reference candidates based on property settings
+   * Delete in-use reference candidates
    *
    * @param GcCandidates Collection of deletion reference candidates to remove.
    */
@@ -140,10 +130,4 @@
    */
   void incrementInUseStat(long i);
 
-  /**
-   * Determine if the given absolute file is still pending replication
-   *
-   * @return True if the file still needs to be replicated
-   */
-  Iterator<Entry<String,Status>> getReplicationNeededIterator();
 }
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 6a7025a..e9c23b1 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -18,8 +18,7 @@
  */
 package org.apache.accumulo.gc;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -28,31 +27,29 @@
 import java.util.stream.IntStream;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
+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.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
 import org.apache.accumulo.core.gc.thrift.GCMonitorService.Iface;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.gc.metrics.GcCycleMetrics;
 import org.apache.accumulo.gc.metrics.GcMetrics;
 import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.rpc.ServerAddress;
@@ -63,6 +60,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
@@ -79,7 +78,7 @@
 
   private final GcCycleMetrics gcCycleMetrics = new GcCycleMetrics();
 
-  SimpleGarbageCollector(ServerOpts opts, String[] args) {
+  SimpleGarbageCollector(ConfigOpts opts, String[] args) {
     super("gc", opts, args);
 
     final AccumuloConfiguration conf = getConfiguration();
@@ -96,7 +95,7 @@
   }
 
   public static void main(String[] args) throws Exception {
-    try (SimpleGarbageCollector gc = new SimpleGarbageCollector(new ServerOpts(), args)) {
+    try (SimpleGarbageCollector gc = new SimpleGarbageCollector(new ConfigOpts(), args)) {
       gc.runServer();
     }
   }
@@ -111,17 +110,6 @@
   }
 
   /**
-   * Checks if the volume manager should move files to the trash rather than delete them.
-   *
-   * @return true if trash is used
-   */
-  boolean isUsingTrash() {
-    @SuppressWarnings("removal")
-    Property p = Property.GC_TRASH_IGNORE;
-    return !getConfiguration().getBoolean(p);
-  }
-
-  /**
    * Gets the number of threads used for deleting files.
    *
    * @return number of delete threads
@@ -169,7 +157,7 @@
     MetricsInfo metricsInfo = getContext().getMetricsInfo();
     metricsInfo.addServiceTags(getApplicationName(), address);
 
-    metricsInfo.addMetricsProducers(new GcMetrics(this));
+    metricsInfo.addMetricsProducers(this, new GcMetrics(this));
     metricsInfo.init();
     try {
       long delay = getStartDelay();
@@ -235,29 +223,11 @@
           log.info(String.format("Collect cycle took %.2f seconds",
               (TimeUnit.NANOSECONDS.toMillis(tStop - tStart) / 1000.0)));
 
-          /*
-           * We want to prune references to fully-replicated WALs from the replication table which
-           * are no longer referenced in the metadata table before running
-           * GarbageCollectWriteAheadLogs to ensure we delete as many files as possible.
-           */
-          Span replSpan = TraceUtil.startSpan(this.getClass(), "replicationClose");
-          try (Scope replScope = replSpan.makeCurrent()) {
-            @SuppressWarnings("deprecation")
-            Runnable closeWals =
-                new org.apache.accumulo.gc.replication.CloseWriteAheadLogReferences(getContext());
-            closeWals.run();
-          } catch (Exception e) {
-            TraceUtil.setException(replSpan, e, false);
-            log.error("Error trying to close write-ahead logs for replication table", e);
-          } finally {
-            replSpan.end();
-          }
-
           // Clean up any unused write-ahead logs
           Span walSpan = TraceUtil.startSpan(this.getClass(), "walogs");
           try (Scope walScope = walSpan.makeCurrent()) {
             GarbageCollectWriteAheadLogs walogCollector =
-                new GarbageCollectWriteAheadLogs(getContext(), fs, liveTServerSet, isUsingTrash());
+                new GarbageCollectWriteAheadLogs(getContext(), fs, liveTServerSet);
             log.info("Beginning garbage collection of write-ahead logs");
             walogCollector.collect(status);
             gcCycleMetrics.setLastWalCollect(status.lastLog);
@@ -285,12 +255,16 @@
 
           switch (action) {
             case "compact":
-              accumuloClient.tableOperations().compact(MetadataTable.NAME, null, null, true, true);
-              accumuloClient.tableOperations().compact(RootTable.NAME, null, null, true, true);
+              accumuloClient.tableOperations().compact(AccumuloTable.METADATA.tableName(), null,
+                  null, true, true);
+              accumuloClient.tableOperations().compact(AccumuloTable.ROOT.tableName(), null, null,
+                  true, true);
               break;
             case "flush":
-              accumuloClient.tableOperations().flush(MetadataTable.NAME, null, null, true);
-              accumuloClient.tableOperations().flush(RootTable.NAME, null, null, true);
+              accumuloClient.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null,
+                  true);
+              accumuloClient.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null,
+                  true);
               break;
             default:
               log.trace("'none - no action' or invalid value provided: {}", action);
@@ -351,9 +325,6 @@
    */
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = getContext().getVolumeManager();
-    if (!isUsingTrash()) {
-      return false;
-    }
     try {
       return fs.moveToTrash(path);
     } catch (FileNotFoundException ex) {
@@ -383,7 +354,7 @@
       ServiceLock lock =
           new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, zooLockUUID);
       if (lock.tryLock(lockWatcher,
-          new ServerServices(addr.toString(), Service.GC_CLIENT).toString().getBytes(UTF_8))) {
+          new ServiceLockData(zooLockUUID, addr.toString(), ThriftService.GC))) {
         log.debug("Got GC ZooKeeper lock");
         return;
       }
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
deleted file mode 100644
index 6cd5cec..0000000
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ /dev/null
@@ -1,206 +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.gc.replication;
-
-import java.time.Duration;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-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.file.rfile.RFile;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.log.WalStateManager;
-import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
-import org.apache.accumulo.server.log.WalStateManager.WalState;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
-
-/**
- * It's impossible to know when all references to a WAL have been removed from the metadata table as
- * the references are potentially spread across the entire tablets row-space.
- * <p>
- * This tool scans the metadata table to collect a set of WALs that are still referenced. Then, each
- * {@link Status} record from the metadata and replication tables that point to that WAL can be
- * "closed", by writing a new Status to the same key with the closed member true.
- */
-@Deprecated
-public class CloseWriteAheadLogReferences implements Runnable {
-  private static final Logger log = LoggerFactory.getLogger(CloseWriteAheadLogReferences.class);
-
-  private static final String RFILE_SUFFIX = "." + RFile.EXTENSION;
-
-  private final ServerContext context;
-
-  public CloseWriteAheadLogReferences(ServerContext context) {
-    this.context = context;
-  }
-
-  @Override
-  public void run() {
-    // Guava Stopwatch is useful here, for a friendlier toString, but the versions of Guava
-    // are different in incompatible ways, so we avoid it here and use Duration instead, so
-    // there won't be conflicts.
-    long startTime;
-    Duration duration;
-
-    if (!ReplicationTable.isOnline(context)) {
-      log.debug("Replication table isn't online, not attempting to clean up wals");
-      return;
-    }
-
-    HashSet<String> closed = null;
-    Span span = TraceUtil.startSpan(this.getClass(), "findReferencedWals");
-    try (Scope findWalsSpan = span.makeCurrent()) {
-      startTime = System.nanoTime();
-      closed = getClosedLogs();
-      duration = Duration.ofNanos(System.nanoTime() - startTime);
-    } finally {
-      span.end();
-    }
-
-    log.info("Found {} WALs referenced in metadata in {}", closed.size(), duration);
-
-    long recordsClosed = 0;
-    Span updateReplicationSpan = TraceUtil.startSpan(this.getClass(), "updateReplicationTable");
-    try (Scope updateReplicationScope = updateReplicationSpan.makeCurrent()) {
-      startTime = System.nanoTime();
-      recordsClosed = updateReplicationEntries(context, closed);
-      duration = Duration.ofNanos(System.nanoTime() - startTime);
-    } finally {
-      updateReplicationSpan.end();
-    }
-
-    log.info("Closed {} WAL replication references in replication table in {}", recordsClosed,
-        duration);
-  }
-
-  /**
-   * Construct the set of referenced WALs from zookeeper
-   *
-   * @return The Set of WALs that are referenced in the metadata table
-   */
-  protected HashSet<String> getClosedLogs() {
-    WalStateManager wals = new WalStateManager(context);
-
-    HashSet<String> result = new HashSet<>();
-    try {
-      for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
-        if (entry.getValue() == WalState.UNREFERENCED || entry.getValue() == WalState.CLOSED) {
-          Path path = entry.getKey();
-          log.debug("Found closed WAL " + path);
-          result.add(path.toString());
-        }
-      }
-    } catch (WalMarkerException e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-
-  /**
-   * Given the set of WALs which have references in the metadata table, close any status messages
-   * with reference that WAL.
-   *
-   * @param client Accumulo client
-   * @param closedWals {@link Set} of paths to WALs that marked as closed or unreferenced in
-   *        zookeeper
-   */
-  protected long updateReplicationEntries(AccumuloClient client, Set<String> closedWals) {
-    long recordsClosed = 0;
-    try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME);
-        BatchScanner bs = client.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)) {
-
-      bs.setRanges(Collections.singleton(Range.prefix(ReplicationSection.getRowPrefix())));
-      bs.fetchColumnFamily(ReplicationSection.COLF);
-
-      Text replFileText = new Text();
-      for (Entry<Key,Value> entry : bs) {
-        Status status;
-        try {
-          status = Status.parseFrom(entry.getValue().get());
-        } catch (InvalidProtocolBufferException e) {
-          log.error("Could not parse Status protobuf for {}", entry.getKey(), e);
-          continue;
-        }
-
-        // Ignore things that aren't completely replicated as we can't delete those anyways
-        ReplicationSection.getFile(entry.getKey(), replFileText);
-        String replFile = replFileText.toString();
-        boolean isClosed = closedWals.contains(replFile);
-
-        // We only want to clean up WALs (which is everything but rfiles) and only when
-        // metadata doesn't have a reference to the given WAL
-        if (!status.getClosed() && !replFile.endsWith(RFILE_SUFFIX) && isClosed) {
-          try {
-            closeWal(bw, entry.getKey());
-            recordsClosed++;
-          } catch (MutationsRejectedException e) {
-            log.error("Failed to submit delete mutation for {}", entry.getKey());
-            continue;
-          }
-        }
-      }
-    } catch (TableNotFoundException e) {
-      log.error("Replication table was deleted", e);
-    } catch (MutationsRejectedException e) {
-      log.error("Failed to write delete mutations for replication table", e);
-    }
-
-    return recordsClosed;
-  }
-
-  /**
-   * Write a closed {@link Status} mutation for the given {@link Key} using the provided
-   * {@link BatchWriter}
-   *
-   * @param bw BatchWriter
-   * @param k Key to create close mutation from
-   */
-  protected void closeWal(BatchWriter bw, Key k) throws MutationsRejectedException {
-    log.debug("Closing unreferenced WAL ({}) in metadata table", k.toStringNoTruncate());
-    Mutation m = new Mutation(k.getRow());
-    m.put(k.getColumnFamily(), k.getColumnQualifier(), StatusUtil.fileClosedValue());
-    bw.addMutation(m);
-  }
-
-}
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
index d9c69da..8f09c4c 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
@@ -20,25 +20,17 @@
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.UUID;
 
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
-import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.replication.ReplicationSchema;
-import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -52,13 +44,6 @@
 
 public class GarbageCollectWriteAheadLogsTest {
 
-  @SuppressWarnings("deprecation")
-  private static final String REPL_TABLE_NAME =
-      org.apache.accumulo.core.replication.ReplicationTable.NAME;
-
-  @SuppressWarnings("deprecation")
-  private static final Text STATUS_SECTION_NAME = ReplicationSchema.StatusSection.NAME;
-
   private final TServerInstance server1 = new TServerInstance("localhost:1234[SESSION]");
   private final TServerInstance server2 = new TServerInstance("localhost:1234[OTHERSESS]");
   private final UUID id = UUID.randomUUID();
@@ -68,16 +53,16 @@
       Collections.singletonMap(server2, Collections.singletonList(id));
   private final Path path = new Path("hdfs://localhost:9000/accumulo/wal/localhost+1234/" + id);
   private final KeyExtent extent = KeyExtent.fromMetaRow(new Text("1<"));
-  private final Collection<Collection<String>> walogs = Collections.emptyList();
+  private final Collection<LogEntry> walogs = Collections.emptyList();
   private final TabletLocationState tabletAssignedToServer1;
   private final TabletLocationState tabletAssignedToServer2;
 
   {
     try {
-      tabletAssignedToServer1 = new TabletLocationState(extent, null, Location.current(server1),
-          null, null, walogs, false);
-      tabletAssignedToServer2 = new TabletLocationState(extent, null, Location.current(server2),
-          null, null, walogs, false);
+      tabletAssignedToServer1 =
+          new TabletLocationState(extent, null, Location.current(server1), null, null, walogs);
+      tabletAssignedToServer2 =
+          new TabletLocationState(extent, null, Location.current(server2), null, null, walogs);
     } catch (Exception ex) {
       throw new RuntimeException(ex);
     }
@@ -87,13 +72,12 @@
       Collections.singletonList(tabletAssignedToServer1);
   private final Iterable<TabletLocationState> tabletOnServer2List =
       Collections.singletonList(tabletAssignedToServer2);
-  private final List<Entry<Key,Value>> emptyList = Collections.emptyList();
-  private final Iterator<Entry<Key,Value>> emptyKV = emptyList.iterator();
 
   @Test
   public void testRemoveUnusedLog() throws Exception {
     ServerContext context = EasyMock.createMock(ServerContext.class);
     VolumeManager fs = EasyMock.createMock(VolumeManager.class);
+    EasyMock.expect(fs.moveToTrash(EasyMock.anyObject())).andReturn(false).anyTimes();
     WalStateManager marker = EasyMock.createMock(WalStateManager.class);
     LiveTServerSet tserverSet = EasyMock.createMock(LiveTServerSet.class);
 
@@ -109,19 +93,13 @@
     marker.removeWalMarker(server1, id);
     EasyMock.expectLastCall().once();
     EasyMock.replay(context, fs, marker, tserverSet);
-    GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, false,
-        tserverSet, marker, tabletOnServer1List) {
-      @Override
-      @Deprecated
-      protected int removeReplicationEntries(Map<UUID,TServerInstance> candidates) {
-        return 0;
-      }
-
-      @Override
-      protected Map<UUID,Path> getSortedWALogs() {
-        return Collections.emptyMap();
-      }
-    };
+    GarbageCollectWriteAheadLogs gc =
+        new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
+          @Override
+          protected Map<UUID,Path> getSortedWALogs() {
+            return Collections.emptyMap();
+          }
+        };
     gc.collect(status);
     EasyMock.verify(context, fs, marker, tserverSet);
   }
@@ -142,19 +120,13 @@
     EasyMock.expect(marker.getAllMarkers()).andReturn(markers).once();
     EasyMock.expect(marker.state(server1, id)).andReturn(new Pair<>(WalState.CLOSED, path));
     EasyMock.replay(context, marker, tserverSet, fs);
-    GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, false,
-        tserverSet, marker, tabletOnServer1List) {
-      @Override
-      @Deprecated
-      protected int removeReplicationEntries(Map<UUID,TServerInstance> candidates) {
-        return 0;
-      }
-
-      @Override
-      protected Map<UUID,Path> getSortedWALogs() {
-        return Collections.emptyMap();
-      }
-    };
+    GarbageCollectWriteAheadLogs gc =
+        new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
+          @Override
+          protected Map<UUID,Path> getSortedWALogs() {
+            return Collections.emptyMap();
+          }
+        };
     gc.collect(status);
     EasyMock.verify(context, marker, tserverSet, fs);
   }
@@ -163,10 +135,9 @@
   public void deleteUnreferenceLogOnDeadServer() throws Exception {
     ServerContext context = EasyMock.createMock(ServerContext.class);
     VolumeManager fs = EasyMock.createMock(VolumeManager.class);
+    EasyMock.expect(fs.moveToTrash(EasyMock.anyObject())).andReturn(false).anyTimes();
     WalStateManager marker = EasyMock.createMock(WalStateManager.class);
     LiveTServerSet tserverSet = EasyMock.createMock(LiveTServerSet.class);
-    Scanner mscanner = EasyMock.createMock(Scanner.class);
-    Scanner rscanner = EasyMock.createMock(Scanner.class);
 
     GCStatus status = new GCStatus(null, null, null, new GcCycleStats());
 
@@ -177,34 +148,21 @@
     EasyMock.expect(marker.getAllMarkers()).andReturn(markers2).once();
     EasyMock.expect(marker.state(server2, id)).andReturn(new Pair<>(WalState.OPEN, path));
 
-    EasyMock.expect(context.createScanner(REPL_TABLE_NAME, Authorizations.EMPTY))
-        .andReturn(rscanner);
-    rscanner.fetchColumnFamily(STATUS_SECTION_NAME);
-    EasyMock.expectLastCall().once();
-    EasyMock.expect(rscanner.iterator()).andReturn(emptyKV);
-
-    EasyMock.expect(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY))
-        .andReturn(mscanner);
-    mscanner.fetchColumnFamily(ReplicationSection.COLF);
-    EasyMock.expectLastCall().once();
-    mscanner.setRange(ReplicationSection.getRange());
-    EasyMock.expectLastCall().once();
-    EasyMock.expect(mscanner.iterator()).andReturn(emptyKV);
     EasyMock.expect(fs.deleteRecursively(path)).andReturn(true).once();
     marker.removeWalMarker(server2, id);
     EasyMock.expectLastCall().once();
     marker.forget(server2);
     EasyMock.expectLastCall().once();
-    EasyMock.replay(context, fs, marker, tserverSet, rscanner, mscanner);
-    GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, false,
-        tserverSet, marker, tabletOnServer1List) {
-      @Override
-      protected Map<UUID,Path> getSortedWALogs() {
-        return Collections.emptyMap();
-      }
-    };
+    EasyMock.replay(context, fs, marker, tserverSet);
+    GarbageCollectWriteAheadLogs gc =
+        new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
+          @Override
+          protected Map<UUID,Path> getSortedWALogs() {
+            return Collections.emptyMap();
+          }
+        };
     gc.collect(status);
-    EasyMock.verify(context, fs, marker, tserverSet, rscanner, mscanner);
+    EasyMock.verify(context, fs, marker, tserverSet);
   }
 
   @Test
@@ -213,8 +171,6 @@
     VolumeManager fs = EasyMock.createMock(VolumeManager.class);
     WalStateManager marker = EasyMock.createMock(WalStateManager.class);
     LiveTServerSet tserverSet = EasyMock.createMock(LiveTServerSet.class);
-    Scanner mscanner = EasyMock.createMock(Scanner.class);
-    Scanner rscanner = EasyMock.createMock(Scanner.class);
 
     GCStatus status = new GCStatus(null, null, null, new GcCycleStats());
 
@@ -225,76 +181,16 @@
     EasyMock.expect(marker.getAllMarkers()).andReturn(markers2).once();
     EasyMock.expect(marker.state(server2, id)).andReturn(new Pair<>(WalState.OPEN, path));
 
-    EasyMock.expect(context.createScanner(REPL_TABLE_NAME, Authorizations.EMPTY))
-        .andReturn(rscanner);
-    rscanner.fetchColumnFamily(STATUS_SECTION_NAME);
-    EasyMock.expectLastCall().once();
-    EasyMock.expect(rscanner.iterator()).andReturn(emptyKV);
-
-    EasyMock.expect(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY))
-        .andReturn(mscanner);
-    mscanner.fetchColumnFamily(ReplicationSection.COLF);
-    EasyMock.expectLastCall().once();
-    mscanner.setRange(ReplicationSection.getRange());
-    EasyMock.expectLastCall().once();
-    EasyMock.expect(mscanner.iterator()).andReturn(emptyKV);
-    EasyMock.replay(context, fs, marker, tserverSet, rscanner, mscanner);
-    GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, false,
-        tserverSet, marker, tabletOnServer2List) {
-      @Override
-      protected Map<UUID,Path> getSortedWALogs() {
-        return Collections.emptyMap();
-      }
-    };
+    EasyMock.replay(context, fs, marker, tserverSet);
+    GarbageCollectWriteAheadLogs gc =
+        new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer2List) {
+          @Override
+          protected Map<UUID,Path> getSortedWALogs() {
+            return Collections.emptyMap();
+          }
+        };
     gc.collect(status);
-    EasyMock.verify(context, fs, marker, tserverSet, rscanner, mscanner);
+    EasyMock.verify(context, fs, marker, tserverSet);
   }
 
-  @Test
-  public void replicationDelaysFileCollection() throws Exception {
-    ServerContext context = EasyMock.createMock(ServerContext.class);
-    VolumeManager fs = EasyMock.createMock(VolumeManager.class);
-    WalStateManager marker = EasyMock.createMock(WalStateManager.class);
-    LiveTServerSet tserverSet = EasyMock.createMock(LiveTServerSet.class);
-    Scanner mscanner = EasyMock.createMock(Scanner.class);
-    Scanner rscanner = EasyMock.createMock(Scanner.class);
-    String row = ReplicationSection.getRowPrefix() + path;
-    String colf = ReplicationSection.COLF.toString();
-    String colq = "1";
-    Map<Key,Value> replicationWork =
-        Collections.singletonMap(new Key(row, colf, colq), new Value());
-
-    GCStatus status = new GCStatus(null, null, null, new GcCycleStats());
-
-    tserverSet.scanServers();
-    EasyMock.expectLastCall();
-    EasyMock.expect(tserverSet.getCurrentServers()).andReturn(Collections.singleton(server1));
-
-    EasyMock.expect(marker.getAllMarkers()).andReturn(markers).once();
-    EasyMock.expect(marker.state(server1, id)).andReturn(new Pair<>(WalState.UNREFERENCED, path));
-
-    EasyMock.expect(context.createScanner(REPL_TABLE_NAME, Authorizations.EMPTY))
-        .andReturn(rscanner);
-    rscanner.fetchColumnFamily(STATUS_SECTION_NAME);
-    EasyMock.expectLastCall().once();
-    EasyMock.expect(rscanner.iterator()).andReturn(emptyKV);
-
-    EasyMock.expect(context.createScanner(MetadataTable.NAME, Authorizations.EMPTY))
-        .andReturn(mscanner);
-    mscanner.fetchColumnFamily(ReplicationSection.COLF);
-    EasyMock.expectLastCall().once();
-    mscanner.setRange(ReplicationSection.getRange());
-    EasyMock.expectLastCall().once();
-    EasyMock.expect(mscanner.iterator()).andReturn(replicationWork.entrySet().iterator());
-    EasyMock.replay(context, fs, marker, tserverSet, rscanner, mscanner);
-    GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, false,
-        tserverSet, marker, tabletOnServer1List) {
-      @Override
-      protected Map<UUID,Path> getSortedWALogs() {
-        return Collections.emptyMap();
-      }
-    };
-    gc.collect(status);
-    EasyMock.verify(context, fs, marker, tserverSet, rscanner, mscanner);
-  }
 }
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
index 8e9a2d1..4c459f1 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
@@ -30,7 +30,6 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -44,11 +43,10 @@
 import org.apache.accumulo.core.gc.ReferenceDirectory;
 import org.apache.accumulo.core.gc.ReferenceFile;
 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 org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.GcCandidateType;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.Test;
 
 public class GarbageCollectionTest {
@@ -62,9 +60,6 @@
 
     ArrayList<GcCandidate> fileDeletions = new ArrayList<>();
     ArrayList<TableId> tablesDirsToDelete = new ArrayList<>();
-    TreeMap<String,Status> filesToReplicate = new TreeMap<>();
-    boolean deleteInUseRefs = false;
-
     private long timestamp = 0L;
 
     private final Ample.DataLevel level;
@@ -94,11 +89,6 @@
     }
 
     @Override
-    public boolean canRemoveInUseCandidates() {
-      return deleteInUseRefs;
-    }
-
-    @Override
     public List<GcCandidate> readCandidatesThatFitInMemory(Iterator<GcCandidate> candidatesIter) {
       List<GcCandidate> candidatesBatch = new ArrayList<>();
       while (candidatesIter.hasNext() && candidatesBatch.size() < 3) {
@@ -133,7 +123,7 @@
     @Override
     public Map<TableId,TableState> getTableIDs() {
       HashMap<TableId,TableState> results = new HashMap<>();
-      tableIds.forEach((t) -> results.put(t, TableState.ONLINE));
+      tableIds.forEach(t -> results.put(t, TableState.ONLINE));
       return results;
     }
 
@@ -150,7 +140,8 @@
 
     public void addFileReference(String tableId, String endRow, String file) {
       TableId tid = TableId.of(tableId);
-      references.put(tableId + ":" + endRow + ":" + file, ReferenceFile.forFile(tid, file));
+      references.put(tableId + ":" + endRow + ":" + file,
+          ReferenceFile.forFile(tid, new Path(file)));
       tableIds.add(tid);
     }
 
@@ -172,7 +163,8 @@
 
     public void addScanReference(String tableId, String endRow, String scan) {
       TableId tid = TableId.of(tableId);
-      references.put(tableId + ":" + endRow + ":scan:" + scan, ReferenceFile.forScan(tid, scan));
+      references.put(tableId + ":" + endRow + ":scan:" + scan,
+          ReferenceFile.forScan(tid, new Path(scan)));
       tableIds.add(tid);
     }
 
@@ -202,16 +194,11 @@
     public void incrementInUseStat(long i) {}
 
     @Override
-    public Iterator<Entry<String,Status>> getReplicationNeededIterator() {
-      return filesToReplicate.entrySet().iterator();
-    }
-
-    @Override
     public Set<TableId> getCandidateTableIDs() {
       if (level == Ample.DataLevel.ROOT) {
-        return Set.of(RootTable.ID);
+        return Set.of(AccumuloTable.ROOT.tableId());
       } else if (level == Ample.DataLevel.METADATA) {
-        return Collections.singleton(MetadataTable.ID);
+        return Collections.singleton(AccumuloTable.METADATA.tableId());
       } else if (level == Ample.DataLevel.USER) {
         Set<TableId> tableIds = new HashSet<>();
         getTableIDs().forEach((k, v) -> {
@@ -221,8 +208,8 @@
             tableIds.add(k);
           }
         });
-        tableIds.remove(MetadataTable.ID);
-        tableIds.remove(RootTable.ID);
+        tableIds.remove(AccumuloTable.METADATA.tableId());
+        tableIds.remove(AccumuloTable.ROOT.tableId());
         return tableIds;
       } else {
         throw new IllegalArgumentException("unknown level " + level);
@@ -239,8 +226,7 @@
   }
 
   private void assertNoCandidatesRemoved(TestGCE gce) {
-    assertEquals(0, gce.deletedCandidates.size(),
-        "Deleted Candidates not empty: " + gce.deleteInUseRefs);
+    assertEquals(0, gce.deletedCandidates.size(), "Deleted Candidates not empty");
   }
 
   private void assertCandidateRemoved(TestGCE gce, GcCandidateType gcCandidateType,
@@ -248,8 +234,7 @@
     for (GcCandidate gcCandidate : gcCandidates) {
       assertEquals(gcCandidateType, gce.deletedCandidates.remove(gcCandidate));
     }
-    assertEquals(0, gce.deletedCandidates.size(),
-        "Deleted Candidates not empty: " + gce.deleteInUseRefs);
+    assertEquals(0, gce.deletedCandidates.size(), "Deleted Candidates not empty");
   }
 
   // This test was created to help track down a ConcurrentModificationException error that was
@@ -280,7 +265,7 @@
 
     var candOne = gce.addCandidate("hdfs://foo:6000/accumulo/tables/4/t0/F000.rf");
     var candTwo = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F001.rf");
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/5/t0/F005.rf");
+    var candThree = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/5/t0/F005.rf");
 
     gce.addFileReference("4", null, "hdfs://foo.com:6000/accumulo/tables/4/t0/F000.rf");
     gce.addFileReference("4", null, "hdfs://foo:6000/accumulo/tables/4/t0/F001.rf");
@@ -291,10 +276,12 @@
 
     gca.collect(gce);
     assertFileDeleted(gce);
+    assertCandidateRemoved(gce, GcCandidateType.INUSE, candOne, candTwo, candThree);
 
     // Remove the reference to this flush file, run the GC which should not trim it from the
     // candidates, and assert that it's gone
     gce.removeFileReference("4", null, "hdfs://foo.com:6000/accumulo/tables/4/t0/F000.rf");
+    candOne = gce.addCandidate("hdfs://foo:6000/accumulo/tables/4/t0/F000.rf");
     gca.collect(gce);
     assertFileDeleted(gce, candOne);
 
@@ -303,16 +290,11 @@
     gca.collect(gce);
     assertFileDeleted(gce);
 
-    // Remove the reference to a file in the candidates should cause it to be removed
-    gce.removeFileReference("4", null, "hdfs://foo:6000/accumulo/tables/4/t0/F001.rf");
+    // Adding more candidates which do not have references that should be removed
+    var candFour = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F003.rf");
+    var candFive = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F004.rf");
     gca.collect(gce);
-    assertFileDeleted(gce, candTwo);
-
-    // Adding more candidates which do not have references should be removed
-    var candThree = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F003.rf");
-    var candFour = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F004.rf");
-    gca.collect(gce);
-    assertFileDeleted(gce, candThree, candFour);
+    assertFileDeleted(gce, candFour, candFive);
 
   }
 
@@ -329,7 +311,7 @@
 
     var candOne = gce.addCandidate("hdfs://foo:6000/accumulo/tables/4/t0/F000.rf");
     var candTwo = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F001.rf");
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/5/t0/F005.rf");
+    var candThree = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/5/t0/F005.rf");
 
     int counter = 0;
     // items to be removed from candidates
@@ -368,10 +350,12 @@
 
     gca.collect(gce);
     assertFileDeleted(gce, toBeRemoved);
+    assertCandidateRemoved(gce, GcCandidateType.INUSE, candOne, candTwo, candThree);
 
-    // Remove the reference to this flush file, run the GC which should not trim it from the
-    // candidates, and assert that it's gone
+    // Remove the reference to this flush file, add the candidate, run the GC and assert that it's
+    // gone
     gce.removeFileReference("4", null, "hdfs://foo.com:6000/accumulo/tables/4/t0/F000.rf");
+    candOne = gce.addCandidate("hdfs://foo:6000/accumulo/tables/4/t0/F000.rf");
     gca.collect(gce);
     assertFileDeleted(gce, candOne);
 
@@ -382,14 +366,15 @@
 
     // Remove the reference to a file in the candidates should cause it to be removed
     gce.removeFileReference("4", null, "hdfs://foo:6000/accumulo/tables/4/t0/F001.rf");
+    candTwo = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F001.rf");
     gca.collect(gce);
     assertFileDeleted(gce, candTwo);
 
     // Adding more candidates which do no have references should be removed
-    var candThree = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F003.rf");
-    var candFour = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F004.rf");
+    var candFour = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F003.rf");
+    var candFive = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F004.rf");
     gca.collect(gce);
-    assertFileDeleted(gce, candThree, candFour);
+    assertFileDeleted(gce, candFour, candFive);
   }
 
   /**
@@ -430,10 +415,6 @@
 
     GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
 
-    // All candidates currently have references
-    gca.collect(gce);
-    assertFileDeleted(gce);
-
     List<String[]> refsToRemove = new ArrayList<>();
     refsToRemove.add(new String[] {"4", "/t0/F000.rf"});
     refsToRemove.add(new String[] {"5", "../4/t0/F000.rf"});
@@ -446,24 +427,33 @@
       gca.collect(gce);
       assertFileDeleted(gce);
     }
+    // InUse references have been removed
+    assertCandidateRemoved(gce, GcCandidateType.INUSE, candOne, candTwo, candThree);
 
     gce.removeFileReference(refsToRemove.get(2)[0], null, refsToRemove.get(2)[1]);
+    var cand = gce.addCandidate(refsToRemove.get(2)[1]);
     gca.collect(gce);
-    assertFileDeleted(gce, candOne);
+    assertFileDeleted(gce, cand);
 
     gce.removeFileReference("4", null, "/t0/F001.rf");
+    candThree = gce.addCandidate("/4/t0/F001.rf");
     gca.collect(gce);
     assertFileDeleted(gce, candThree);
 
-    // add absolute candidate for file that already has a relative candidate
+    // add an absolute candidate for file that already has a relative candidate
     var candFour = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F002.rf");
     gca.collect(gce);
     assertFileDeleted(gce);
+    assertCandidateRemoved(gce, GcCandidateType.INUSE, candFour);
 
+    // Re-add the absolute candidate
+    candFour = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/4/t0/F002.rf");
     gce.removeFileReference("4", null, "/t0/F002.rf");
     gca.collect(gce);
     assertFileDeleted(gce, candFour);
 
+    // Finally re-add the relative candidate to remove the last file
+    candTwo = gce.addCandidate("/4/t0/F002.rf");
     gca.collect(gce);
     assertFileDeleted(gce, candTwo);
   }
@@ -809,90 +799,6 @@
   }
 
   @Test
-  public void finishedReplicationRecordsDontPreventDeletion() throws Exception {
-    GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-
-    TestGCE gce = new TestGCE();
-
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf");
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/2/t-00002/A000002.rf");
-
-    Status status = Status.newBuilder().setClosed(true).setEnd(100).setBegin(100).build();
-    gce.filesToReplicate.put("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf", status);
-
-    gca.collect(gce);
-
-    // No refs to A000002.rf, and a closed, finished repl for A000001.rf should not preclude
-    // it from being deleted
-    assertEquals(2, gce.fileDeletions.size());
-  }
-
-  @Test
-  public void openReplicationRecordsPreventDeletion() throws Exception {
-    GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-
-    TestGCE gce = new TestGCE();
-
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf");
-    var candidate = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/2/t-00002/A000002.rf");
-
-    // We replicated all of the data, but we might still write more data to the file
-    Status status = Status.newBuilder().setClosed(false).setEnd(1000).setBegin(100).build();
-    gce.filesToReplicate.put("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf", status);
-
-    gca.collect(gce);
-
-    // We need to replicate that one file still, should not delete it.
-    assertEquals(1, gce.fileDeletions.size());
-    assertEquals(candidate, gce.fileDeletions.get(0));
-  }
-
-  @Test
-  public void newReplicationRecordsPreventDeletion() throws Exception {
-    GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-
-    TestGCE gce = new TestGCE();
-
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf");
-    var candidate = gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/2/t-00002/A000002.rf");
-
-    // We replicated all of the data, but we might still write more data to the file
-    @SuppressWarnings("deprecation")
-    Status status =
-        org.apache.accumulo.server.replication.StatusUtil.fileCreated(System.currentTimeMillis());
-    gce.filesToReplicate.put("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf", status);
-
-    gca.collect(gce);
-
-    // We need to replicate that one file still, should not delete it.
-    assertEquals(1, gce.fileDeletions.size());
-    assertEquals(candidate, gce.fileDeletions.get(0));
-  }
-
-  @Test
-  public void bulkImportReplicationRecordsPreventDeletion() throws Exception {
-    GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-
-    TestGCE gce = new TestGCE();
-
-    assertEquals(0, gce.fileDeletions.size());
-
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf");
-    gce.addCandidate("hdfs://foo.com:6000/accumulo/tables/2/t-00002/A000002.rf");
-
-    // Some file of unknown length has no replication yet (representative of the bulk-import case)
-    Status status = Status.newBuilder().setInfiniteEnd(true).setBegin(0).setClosed(true).build();
-    gce.filesToReplicate.put("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf", status);
-
-    gca.collect(gce);
-
-    // We need to replicate that one file still, should not delete it.
-    assertEquals(1, gce.fileDeletions.size());
-    assertEquals(new GcCandidate("hdfs://foo.com:6000/accumulo/tables/2/t-00002/A000002.rf", 1L),
-        gce.fileDeletions.get(0));
-  }
-
-  @Test
   public void testMissingTableIds() throws Exception {
     GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
 
@@ -926,16 +832,9 @@
     gce.addFileReference("6", null, "hdfs://foo.com:6000/accumulo/tables/4/t0/F000.rf");
 
     GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-    gce.deleteInUseRefs = false;
     // All candidates currently have references
     gca.collect(gce);
     assertFileDeleted(gce);
-    assertNoCandidatesRemoved(gce);
-
-    // Enable InUseRefs to be removed if the file ref is found.
-    gce.deleteInUseRefs = true;
-    gca.collect(gce);
-    assertFileDeleted(gce);
     assertCandidateRemoved(gce, GcCandidateType.INUSE, candidate);
 
     var cand1 = gce.addCandidate("/9/t0/F003.rf");
@@ -969,17 +868,9 @@
     gce.addFileReference("+r", null, "hdfs://foo.com:6000/accumulo/tables/4/t0/F000.rf");
 
     GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-    gce.deleteInUseRefs = false;
-    // No InUse Candidates should be removed.
     gca.collect(gce);
     assertFileDeleted(gce);
-    assertNoCandidatesRemoved(gce);
-
-    gce.deleteInUseRefs = true;
-    // Due to the gce Datalevel of ROOT, InUse candidate deletion is not supported regardless of
-    // property setting.
-    gca.collect(gce);
-    assertFileDeleted(gce);
+    // Due to the gce Datalevel of ROOT, InUse candidate deletion is not supported
     assertNoCandidatesRemoved(gce);
 
     gce.removeFileReference("+r", null, "/t0/F000.rf");
@@ -998,7 +889,7 @@
     TestGCE gce = new TestGCE();
     GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
 
-    // Check expected starting state.
+    // Check the expected starting state.
     assertEquals(0, gce.candidates.size());
 
     // Ensure that dir candidates still work
@@ -1020,14 +911,11 @@
 
     assertEquals(0, gce.candidates.size());
 
-    // Now enable InUse deletions
-    gce.deleteInUseRefs = true;
-
     // Add deletion candidate for a directory.
     var candidate = new GcCandidate("6/t-0/", 10L);
     gce.candidates.add(candidate);
 
-    // Then create a InUse candidate for a file in that directory.
+    // Then create an InUse candidate for a file in that directory.
     gce.addFileReference("6", null, "/t-0/F003.rf");
     var removedCandidate = gce.addCandidate("6/t-0/F003.rf");
 
@@ -1051,8 +939,6 @@
     gce.addFileReference("4", null, "/t0/F000.rf");
 
     GarbageCollectionAlgorithm gca = new GarbageCollectionAlgorithm();
-    gce.deleteInUseRefs = true;
-
     gca.collect(gce);
     assertFileDeleted(gce, candTwo);
     assertCandidateRemoved(gce, GcCandidateType.INUSE, candOne);
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
index fdb66ba..b5a7ea3 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
@@ -66,8 +66,6 @@
   private SimpleGarbageCollector gc;
   private ConfigurationCopy systemConfig;
   private static SiteConfiguration siteConfig = SiteConfiguration.empty().build();
-  @SuppressWarnings("removal")
-  private final Property GC_TRASH_IGNORE = Property.GC_TRASH_IGNORE;
 
   @BeforeEach
   public void setUp() {
@@ -100,7 +98,6 @@
     conf.put(Property.GC_CYCLE_START.getKey(), "1");
     conf.put(Property.GC_CYCLE_DELAY.getKey(), "20");
     conf.put(Property.GC_DELETE_THREADS.getKey(), "2");
-    conf.put(GC_TRASH_IGNORE.getKey(), "false");
 
     return new ConfigurationCopy(conf);
   }
@@ -109,7 +106,6 @@
   public void testInit() {
     assertSame(volMgr, gc.getContext().getVolumeManager());
     assertEquals(credentials, gc.getContext().getCredentials());
-    assertTrue(gc.isUsingTrash());
     assertEquals(1000L, gc.getStartDelay());
     assertEquals(2, gc.getNumDeleteThreads());
     assertFalse(gc.inSafeMode()); // false by default
@@ -134,13 +130,6 @@
   }
 
   @Test
-  public void testMoveToTrash_NotUsingTrash() throws Exception {
-    systemConfig.set(GC_TRASH_IGNORE.getKey(), "true");
-    Path path = createMock(Path.class);
-    assertFalse(gc.moveToTrash(path));
-  }
-
-  @Test
   public void testIsDir() {
     assertTrue(SimpleGarbageCollector.isDir("tid1/dir1"));
     assertTrue(SimpleGarbageCollector.isDir("/dir1"));
@@ -178,7 +167,7 @@
     confirmed.put("5a/t-0002/F0001.rf",
         new GcCandidate("hdfs://nn1/accumulo/tables/5a/t-0002/F0001.rf", 3L));
     var allVolumesDirectory = new AllVolumesDirectory(TableId.of("5b"), "t-0003");
-    confirmed.put("5b/t-0003", new GcCandidate(allVolumesDirectory.getMetadataEntry(), 4L));
+    confirmed.put("5b/t-0003", new GcCandidate(allVolumesDirectory.getMetadataPath(), 4L));
     confirmed.put("5b/t-0003/F0001.rf",
         new GcCandidate("hdfs://nn1/accumulo/tables/5b/t-0003/F0001.rf", 5L));
     confirmed.put("5b/t-0003/F0002.rf",
@@ -186,7 +175,7 @@
     confirmed.put("5b/t-0003/F0003.rf",
         new GcCandidate("hdfs://nn3/accumulo/tables/5b/t-0003/F0003.rf", 7L));
     allVolumesDirectory = new AllVolumesDirectory(TableId.of("5b"), "t-0004");
-    confirmed.put("5b/t-0004", new GcCandidate(allVolumesDirectory.getMetadataEntry(), 8L));
+    confirmed.put("5b/t-0004", new GcCandidate(allVolumesDirectory.getMetadataPath(), 8L));
     confirmed.put("5b/t-0004/F0001.rf",
         new GcCandidate("hdfs://nn1/accumulo/tables/5b/t-0004/F0001.rf", 9L));
 
@@ -199,11 +188,11 @@
     expected.put("5a/t-0002/F0001.rf",
         new GcCandidate("hdfs://nn1/accumulo/tables/5a/t-0002/F0001.rf", 3L));
     allVolumesDirectory = new AllVolumesDirectory(TableId.of("5b"), "t-0003");
-    expected.put("5b/t-0003", new GcCandidate(allVolumesDirectory.getMetadataEntry(), 4L));
+    expected.put("5b/t-0003", new GcCandidate(allVolumesDirectory.getMetadataPath(), 4L));
     expected.put("5b/t-0003/F0003.rf",
         new GcCandidate("hdfs://nn3/accumulo/tables/5b/t-0003/F0003.rf", 7L));
     allVolumesDirectory = new AllVolumesDirectory(TableId.of("5b"), "t-0004");
-    expected.put("5b/t-0004", new GcCandidate(allVolumesDirectory.getMetadataEntry(), 8L));
+    expected.put("5b/t-0004", new GcCandidate(allVolumesDirectory.getMetadataPath(), 8L));
 
     assertEquals(expected, confirmed);
     assertEquals(
diff --git a/server/manager/pom.xml b/server/manager/pom.xml
index 920509a..0213cf2 100644
--- a/server/manager/pom.xml
+++ b/server/manager/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-manager</artifactId>
@@ -32,6 +32,10 @@
   <description>The manager server for Apache Accumulo for load balancing and other system-wide operations.</description>
   <dependencies>
     <dependency>
+      <groupId>com.github.ben-manes.caffeine</groupId>
+      <artifactId>caffeine</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.auto.service</groupId>
       <artifactId>auto-service</artifactId>
       <optional>true</optional>
@@ -45,10 +49,6 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
@@ -101,6 +101,10 @@
       <artifactId>zookeeper-jute</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.checkerframework</groupId>
+      <artifactId>checker-qual</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
@@ -124,5 +128,10 @@
       <artifactId>junit-jupiter-api</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
index 0444099..9d66cdc 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
@@ -53,6 +53,7 @@
 import org.apache.accumulo.core.clientImpl.TableOperationsImpl;
 import org.apache.accumulo.core.clientImpl.UserCompactionUtils;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftNotActiveServiceException;
@@ -63,12 +64,11 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
 import org.apache.accumulo.core.manager.thrift.FateOperation;
 import org.apache.accumulo.core.manager.thrift.FateService;
 import org.apache.accumulo.core.manager.thrift.ThriftPropertyException;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.accumulo.core.util.Validator;
@@ -493,41 +493,6 @@
             autoCleanup, goalMessage);
         break;
       }
-      case TABLE_BULK_IMPORT: {
-        TableOperation tableOp = TableOperation.BULK_IMPORT;
-        validateArgumentCount(arguments, tableOp, 4);
-        String tableName =
-            validateName(arguments.get(0), tableOp, EXISTING_TABLE_NAME.and(NOT_BUILTIN_TABLE));
-        String dir = ByteBufferUtil.toString(arguments.get(1));
-        String failDir = ByteBufferUtil.toString(arguments.get(2));
-        boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
-
-        final TableId tableId =
-            ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
-        NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
-
-        final boolean canBulkImport;
-        try {
-          canBulkImport =
-              manager.security.canBulkImport(c, tableId, tableName, dir, failDir, namespaceId);
-        } catch (ThriftSecurityException e) {
-          throwIfTableMissingSecurityException(e, tableId, tableName, TableOperation.BULK_IMPORT);
-          throw e;
-        }
-
-        if (!canBulkImport) {
-          throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-        }
-
-        manager.updateBulkImportStatus(dir, BulkImportState.INITIAL);
-        goalMessage +=
-            "Bulk import " + dir + " to " + tableName + "(" + tableId + ") failing to " + failDir;
-        manager.fate().seedTransaction(op.toString(), opid,
-            new TraceRepo<>(new org.apache.accumulo.manager.tableOps.bulkVer1.BulkImport(tableId,
-                dir, failDir, setTime)),
-            autoCleanup, goalMessage);
-        break;
-      }
       case TABLE_COMPACT: {
         TableOperation tableOp = TableOperation.COMPACT;
         validateArgumentCount(arguments, tableOp, 2);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index 86a1dd7..ded6d62 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.manager;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Collections.emptySortedMap;
 import static java.util.concurrent.TimeUnit.HOURS;
@@ -25,10 +26,10 @@
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.net.UnknownHostException;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -54,6 +55,7 @@
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -69,36 +71,35 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.AgeOffStore;
 import org.apache.accumulo.core.fate.Fate;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.manager.balancer.AssignmentParamsImpl;
 import org.apache.accumulo.core.manager.balancer.BalanceParamsImpl;
 import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
 import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
-import org.apache.accumulo.core.metadata.TabletState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.metrics.MetricsProducer;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
 import org.apache.accumulo.core.spi.balancer.SimpleLoadBalancer;
@@ -106,21 +107,23 @@
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
-import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
+import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
+import org.apache.accumulo.core.util.time.NanoTime;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.accumulo.manager.metrics.ManagerMetrics;
 import org.apache.accumulo.manager.recovery.RecoveryManager;
 import org.apache.accumulo.manager.state.TableCounts;
 import org.apache.accumulo.manager.tableOps.TraceRepo;
+import org.apache.accumulo.manager.upgrade.PreUpgradeValidation;
 import org.apache.accumulo.manager.upgrade.UpgradeCoordinator;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
@@ -156,8 +159,10 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Comparators;
 import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.util.concurrent.RateLimiter;
+import com.google.common.util.concurrent.Uninterruptibles;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import io.opentelemetry.api.trace.Span;
@@ -194,8 +199,6 @@
       Collections.synchronizedSortedMap(new TreeMap<>());
   final EventCoordinator nextEvent = new EventCoordinator();
   private final Object mergeLock = new Object();
-  private Thread replicationWorkThread;
-  private Thread replicationAssignerThread;
   RecoveryManager recoveryManager = null;
   private final ManagerTime timeKeeper;
 
@@ -225,6 +228,8 @@
   private final AtomicBoolean managerUpgrading = new AtomicBoolean(false);
   private final long timeToCacheRecoveryWalExistence;
 
+  private ExecutorService tableInformationStatusPool = null;
+
   @Override
   public synchronized ManagerState getManagerState() {
     return state;
@@ -303,6 +308,7 @@
     }
 
     if (oldState != newState && (newState == ManagerState.HAVE_LOCK)) {
+      new PreUpgradeValidation().validate(getContext(), nextEvent);
       upgradeCoordinator.upgradeZookeeper(getContext(), nextEvent);
     }
 
@@ -343,8 +349,8 @@
   }
 
   private int nonMetaDataTabletsAssignedOrHosted() {
-    return totalAssignedOrHosted() - assignedOrHosted(MetadataTable.ID)
-        - assignedOrHosted(RootTable.ID);
+    return totalAssignedOrHosted() - assignedOrHosted(AccumuloTable.METADATA.tableId())
+        - assignedOrHosted(AccumuloTable.ROOT.tableId());
   }
 
   private int notHosted() {
@@ -378,14 +384,14 @@
       case SAFE_MODE:
         // Count offline tablets for the metadata table
         for (TabletGroupWatcher watcher : watchers) {
-          TableCounts counts = watcher.getStats(MetadataTable.ID);
+          TableCounts counts = watcher.getStats(AccumuloTable.METADATA.tableId());
           result += counts.unassigned() + counts.suspended();
         }
         break;
       case UNLOAD_METADATA_TABLETS:
       case UNLOAD_ROOT_TABLET:
         for (TabletGroupWatcher watcher : watchers) {
-          TableCounts counts = watcher.getStats(MetadataTable.ID);
+          TableCounts counts = watcher.getStats(AccumuloTable.METADATA.tableId());
           result += counts.unassigned() + counts.suspended();
         }
         break;
@@ -409,12 +415,12 @@
   }
 
   public static void main(String[] args) throws Exception {
-    try (Manager manager = new Manager(new ServerOpts(), args)) {
+    try (Manager manager = new Manager(new ConfigOpts(), args)) {
       manager.runServer();
     }
   }
 
-  Manager(ServerOpts opts, String[] args) throws IOException {
+  Manager(ConfigOpts opts, String[] args) throws IOException {
     super("manager", opts, args);
     ServerContext context = super.getContext();
     balancerEnvironment = new BalancerEnvironmentImpl(context);
@@ -645,17 +651,6 @@
             case COMPLETE:
               break;
             case STARTED:
-            case SPLITTING:
-              return TabletGoalState.HOSTED;
-            case WAITING_FOR_CHOPPED:
-              if (tls.getState(tserverSet.getCurrentServers()).equals(TabletState.HOSTED)) {
-                if (tls.chopped) {
-                  return TabletGoalState.UNASSIGNED;
-                }
-              } else if (tls.chopped && tls.walogs.isEmpty()) {
-                return TabletGoalState.UNASSIGNED;
-              }
-
               return TabletGoalState.HOSTED;
             case WAITING_FOR_OFFLINE:
               // If we have walogs we need to be HOSTED to recover
@@ -665,6 +660,7 @@
                 return TabletGoalState.UNASSIGNED;
               }
             case MERGING:
+            case MERGED:
               return TabletGoalState.UNASSIGNED;
           }
         } else {
@@ -709,7 +705,8 @@
      */
     private void cleanupNonexistentMigrations(final AccumuloClient accumuloClient)
         throws TableNotFoundException {
-      Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      Scanner scanner =
+          accumuloClient.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       scanner.setRange(MetadataSchema.TabletsSection.getRange());
       Set<KeyExtent> notSeen;
@@ -798,7 +795,7 @@
                 }
                   break;
                 case UNLOAD_METADATA_TABLETS: {
-                  int count = assignedOrHosted(MetadataTable.ID);
+                  int count = assignedOrHosted(AccumuloTable.METADATA.tableId());
                   log.debug(
                       String.format("There are %d metadata tablets assigned or hosted", count));
                   if (count == 0 && goodStats()) {
@@ -807,12 +804,12 @@
                 }
                   break;
                 case UNLOAD_ROOT_TABLET:
-                  int count = assignedOrHosted(MetadataTable.ID);
+                  int count = assignedOrHosted(AccumuloTable.METADATA.tableId());
                   if (count > 0 && goodStats()) {
                     log.debug(String.format("%d metadata tablets online", count));
                     setManagerState(ManagerState.UNLOAD_ROOT_TABLET);
                   }
-                  int root_count = assignedOrHosted(RootTable.ID);
+                  int root_count = assignedOrHosted(AccumuloTable.ROOT.tableId());
                   if (root_count > 0 && goodStats()) {
                     log.debug("The root tablet is still assigned or hosted");
                   }
@@ -968,11 +965,10 @@
       Set<TServerInstance> currentServers, SortedMap<TabletServerId,TServerStatus> balancerMap) {
     final long rpcTimeout = getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT);
     int threads = getConfiguration().getCount(Property.MANAGER_STATUS_THREAD_POOL_SIZE);
-    ExecutorService tp = ThreadPools.getServerThreadPools()
-        .createExecutorService(getConfiguration(), Property.MANAGER_STATUS_THREAD_POOL_SIZE);
     long start = System.currentTimeMillis();
     final SortedMap<TServerInstance,TabletServerStatus> result = new ConcurrentSkipListMap<>();
     final RateLimiter shutdownServerRateLimiter = RateLimiter.create(MAX_SHUTDOWNS_PER_SEC);
+    final ArrayList<Future<?>> tasks = new ArrayList<>();
     for (TServerInstance serverInstance : currentServers) {
       final TServerInstance server = serverInstance;
       if (threads == 0) {
@@ -981,7 +977,7 @@
         // unresponsive tservers.
         sleepUninterruptibly(Math.max(1, rpcTimeout / 120_000), MILLISECONDS);
       }
-      tp.execute(() -> {
+      tasks.add(tableInformationStatusPool.submit(() -> {
         try {
           Thread t = Thread.currentThread();
           String oldName = t.getName();
@@ -1030,17 +1026,29 @@
             badServers.remove(server);
           }
         }
-      });
+      }));
     }
-    tp.shutdown();
-    try {
-      tp.awaitTermination(Math.max(10000, rpcTimeout / 3), MILLISECONDS);
-    } catch (InterruptedException e) {
-      log.debug("Interrupted while fetching status");
+    // wait at least 10 seconds
+    final Duration timeToWait =
+        Comparators.max(Duration.ofSeconds(10), Duration.ofMillis(rpcTimeout / 3));
+    final NanoTime startTime = NanoTime.now();
+    // Wait for all tasks to complete
+    while (!tasks.isEmpty()) {
+      boolean cancel = (startTime.elapsed().compareTo(timeToWait) > 0);
+      Iterator<Future<?>> iter = tasks.iterator();
+      while (iter.hasNext()) {
+        Future<?> f = iter.next();
+        if (cancel) {
+          f.cancel(true);
+        } else {
+          if (f.isDone()) {
+            iter.remove();
+          }
+        }
+      }
+      Uninterruptibles.sleepUninterruptibly(1, MILLISECONDS);
     }
 
-    tp.shutdownNow();
-
     // Threads may still modify map after shutdownNow is called, so create an immutable snapshot.
     SortedMap<TServerInstance,TabletServerStatus> info = ImmutableSortedMap.copyOf(result);
     tserverStatus.forEach((tsi, status) -> balancerMap.put(new TabletServerIdImpl(tsi),
@@ -1088,8 +1096,9 @@
     log.info("Started Manager client service at {}", sa.address);
 
     // block until we can obtain the ZK lock for the manager
+    ServiceLockData sld = null;
     try {
-      getManagerLock(ServiceLock.path(zroot + Constants.ZMANAGER_LOCK));
+      sld = getManagerLock(ServiceLock.path(zroot + Constants.ZMANAGER_LOCK));
     } catch (KeeperException | InterruptedException e) {
       throw new IllegalStateException("Exception getting manager lock", e);
     }
@@ -1111,6 +1120,9 @@
 
     context.getTableManager().addObserver(this);
 
+    tableInformationStatusPool = ThreadPools.getServerThreadPools()
+        .createExecutorService(getConfiguration(), Property.MANAGER_STATUS_THREAD_POOL_SIZE, false);
+
     Thread statusThread = Threads.createThread("Status Thread", new StatusThread());
     statusThread.start();
 
@@ -1194,8 +1206,7 @@
               context.getZooReaderWriter()),
           HOURS.toMillis(8), System::currentTimeMillis);
 
-      Fate<Manager> f = new Fate<>(this, store, TraceRepo::toLogString);
-      f.startTransactionRunners(getConfiguration());
+      Fate<Manager> f = new Fate<>(this, store, TraceRepo::toLogString, getConfiguration());
       fateRef.set(f);
       fateReadyLatch.countDown();
 
@@ -1208,8 +1219,6 @@
     ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor()
         .scheduleWithFixedDelay(() -> ScanServerMetadataEntries.clean(context), 10, 10, MINUTES));
 
-    initializeZkForReplication(zReaderWriter, zroot);
-
     // Make sure that we have a secret key (either a new one or an old one from ZK) before we start
     // the manager client service.
     Thread authenticationTokenKeyManagerThread = null;
@@ -1237,9 +1246,11 @@
     }
 
     String address = sa.address.toString();
-    log.info("Setting manager lock data to {}", address);
+    sld = new ServiceLockData(sld.getServerUUID(ThriftService.MANAGER), address,
+        ThriftService.MANAGER);
+    log.info("Setting manager lock data to {}", sld.toString());
     try {
-      managerLock.replaceLockData(address.getBytes(UTF_8));
+      managerLock.replaceLockData(sld);
     } catch (KeeperException | InterruptedException e) {
       throw new IllegalStateException("Exception updating manager lock", e);
     }
@@ -1248,25 +1259,6 @@
       sleepUninterruptibly(100, MILLISECONDS);
     }
 
-    // if the replication name is ever set, then start replication services
-    final AtomicReference<TServer> replServer = new AtomicReference<>();
-    ScheduledFuture<?> future = context.getScheduledExecutor().scheduleWithFixedDelay(() -> {
-      try {
-        @SuppressWarnings("deprecation")
-        Property p = Property.REPLICATION_NAME;
-        if ((replServer.get() == null) && !getConfiguration().get(p).isEmpty()) {
-          log.info("{} was set, starting repl services.", p.getKey());
-          replServer.set(setupReplication());
-        }
-      } catch (UnknownHostException | KeeperException | InterruptedException e) {
-        log.error("Error occurred starting replication services. ", e);
-      }
-    }, 0, 5000, MILLISECONDS);
-    ThreadPools.watchNonCriticalScheduledTask(future);
-
-    // checking stored user hashes if any of them uses an outdated algorithm
-    security.validateStoredUserCreditentials();
-
     // The manager is fully initialized. Clients are allowed to connect now.
     managerInitialized.set(true);
 
@@ -1279,19 +1271,11 @@
     final long deadline = System.currentTimeMillis() + MAX_CLEANUP_WAIT_TIME;
     try {
       statusThread.join(remaining(deadline));
-      if (null != replicationAssignerThread) {
-        replicationAssignerThread.join(remaining(deadline));
-      }
-      if (null != replicationWorkThread) {
-        replicationWorkThread.join(remaining(deadline));
-      }
     } catch (InterruptedException e) {
-      throw new IllegalStateException("Exception stopping replication workers", e);
+      throw new IllegalStateException("Exception stopping status thread", e);
     }
-    var nullableReplServer = replServer.get();
-    if (nullableReplServer != null) {
-      nullableReplServer.stop();
-    }
+
+    tableInformationStatusPool.shutdownNow();
 
     // Signal that we want it to stop, and wait for it to do so.
     if (authenticationTokenKeyManager != null) {
@@ -1317,16 +1301,6 @@
     log.info("exiting");
   }
 
-  @Deprecated
-  private void initializeZkForReplication(ZooReaderWriter zReaderWriter, String zroot) {
-    try {
-      org.apache.accumulo.server.replication.ZooKeeperInitialization
-          .ensureZooKeeperInitialized(zReaderWriter, zroot);
-    } catch (KeeperException | InterruptedException e) {
-      throw new IllegalStateException("Exception while ensuring ZooKeeper is initialized", e);
-    }
-  }
-
   /**
    * Allows property configuration to block manager start-up waiting for a minimum number of
    * tservers to register in zookeeper. It also accepts a maximum time to wait - if the time
@@ -1375,9 +1349,10 @@
       waitIncrement = 5;
     }
 
-    Retry tserverRetry = Retry.builder().maxRetries(retries).retryAfter(initialWait, SECONDS)
-        .incrementBy(waitIncrement, SECONDS).maxWait(maxWaitPeriod, SECONDS).backOffFactor(1)
-        .logInterval(30, SECONDS).createRetry();
+    Retry tserverRetry = Retry.builder().maxRetries(retries)
+        .retryAfter(Duration.ofSeconds(initialWait)).incrementBy(Duration.ofSeconds(waitIncrement))
+        .maxWait(Duration.ofSeconds(maxWaitPeriod)).backOffFactor(1)
+        .logInterval(Duration.ofSeconds(30)).createRetry();
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
@@ -1414,41 +1389,6 @@
     }
   }
 
-  @Deprecated
-  private TServer setupReplication()
-      throws UnknownHostException, KeeperException, InterruptedException {
-    ServerContext context = getContext();
-    // Start the replication coordinator which assigns tservers to service replication requests
-    var impl = new org.apache.accumulo.manager.replication.ManagerReplicationCoordinator(this);
-    ReplicationCoordinator.Iface haReplicationProxy =
-        HighlyAvailableServiceWrapper.service(impl, this);
-
-    var processor =
-        ThriftProcessorTypes.getReplicationCoordinatorTProcessor(haReplicationProxy, getContext());
-
-    ServerAddress replAddress = TServerUtils.startServer(context, getHostname(),
-        Property.MANAGER_REPLICATION_COORDINATOR_PORT, processor, "Manager Replication Coordinator",
-        "Replication Coordinator", null, Property.MANAGER_REPLICATION_COORDINATOR_MINTHREADS, null,
-        Property.MANAGER_REPLICATION_COORDINATOR_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
-
-    log.info("Started replication coordinator service at " + replAddress.address);
-    // Start the daemon to scan the replication table and make units of work
-    replicationWorkThread = Threads.createThread("Replication Driver",
-        new org.apache.accumulo.manager.replication.ReplicationDriver(this));
-    replicationWorkThread.start();
-
-    // Start the daemon to assign work to tservers to replicate to our peers
-    var wd = new org.apache.accumulo.manager.replication.WorkDriver(this);
-    replicationAssignerThread = Threads.createThread(wd.getName(), wd);
-    replicationAssignerThread.start();
-
-    // Advertise that port we used so peers don't have to be told what it is
-    context.getZooReaderWriter().putPersistentData(
-        getZooKeeperRoot() + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR,
-        replAddress.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
-    return replAddress.server;
-  }
-
   private long remaining(long deadline) {
     return Math.max(1, deadline - System.currentTimeMillis());
   }
@@ -1516,7 +1456,7 @@
     }
   }
 
-  private void getManagerLock(final ServiceLockPath zManagerLoc)
+  private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc)
       throws KeeperException, InterruptedException {
     var zooKeeper = getContext().getZooReaderWriter().getZooKeeper();
     log.info("trying to get manager lock");
@@ -1525,11 +1465,13 @@
         getHostname() + ":" + getConfiguration().getPort(Property.MANAGER_CLIENTPORT)[0];
 
     UUID zooLockUUID = UUID.randomUUID();
+    ServiceLockData sld =
+        new ServiceLockData(zooLockUUID, managerClientAddress, ThriftService.MANAGER);
     while (true) {
 
       ManagerLockWatcher managerLockWatcher = new ManagerLockWatcher();
       managerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID);
-      managerLock.lock(managerLockWatcher, managerClientAddress.getBytes(UTF_8));
+      managerLock.lock(managerLockWatcher, sld);
 
       managerLockWatcher.waitForChange();
 
@@ -1547,6 +1489,7 @@
     }
 
     setManagerState(ManagerState.HAVE_LOCK);
+    return sld;
   }
 
   @Override
@@ -1645,10 +1588,10 @@
     Set<TableId> result = new HashSet<>();
     if (getManagerState() != ManagerState.NORMAL) {
       if (getManagerState() != ManagerState.UNLOAD_METADATA_TABLETS) {
-        result.add(MetadataTable.ID);
+        result.add(AccumuloTable.METADATA.tableId());
       }
       if (getManagerState() != ManagerState.UNLOAD_ROOT_TABLET) {
-        result.add(RootTable.ID);
+        result.add(AccumuloTable.ROOT.tableId());
       }
       return result;
     }
@@ -1784,11 +1727,11 @@
   }
 
   /**
-   * Return how long (in milliseconds) there has been a manager overseeing this cluster. This is an
-   * approximately monotonic clock, which will be approximately consistent between different
-   * managers or different runs of the same manager.
+   * Return how long there has been a manager overseeing this cluster. This is an approximately
+   * monotonic clock, which will be approximately consistent between different managers or different
+   * runs of the same manager. SteadyTime supports both nanoseconds and milliseconds.
    */
-  public Long getSteadyTime() {
+  public SteadyTime getSteadyTime() {
     return timeKeeper.getTime();
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
index 1a8d0d0..b05f224 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
@@ -18,16 +18,15 @@
  */
 package org.apache.accumulo.manager;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FLUSH_ID;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.ConcurrentModificationException;
 import java.util.HashSet;
 import java.util.List;
@@ -37,14 +36,13 @@
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.DelegationTokenConfigSerializer;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
@@ -53,11 +51,8 @@
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.NamespaceId;
-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.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.fate.Fate;
@@ -69,19 +64,14 @@
 import org.apache.accumulo.core.manager.thrift.TabletLoadState;
 import org.apache.accumulo.core.manager.thrift.TabletSplit;
 import org.apache.accumulo.core.manager.thrift.ThriftPropertyException;
-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.TServerInstance;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.metadata.schema.TabletDeletedException;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.securityImpl.thrift.TDelegationToken;
 import org.apache.accumulo.core.securityImpl.thrift.TDelegationTokenConfig;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.manager.tableOps.TraceRepo;
 import org.apache.accumulo.manager.tserverOps.ShutdownTServer;
@@ -89,7 +79,6 @@
 import org.apache.accumulo.server.conf.store.NamespacePropKey;
 import org.apache.accumulo.server.conf.store.TablePropKey;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.security.delegation.AuthenticationTokenSecretManager;
 import org.apache.accumulo.server.util.PropUtil;
 import org.apache.accumulo.server.util.SystemPropUtil;
@@ -98,15 +87,10 @@
 import org.apache.thrift.TException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
 
 public class ManagerClientServiceHandler implements ManagerClientService.Iface {
 
   private static final Logger log = Manager.log;
-  private static final Logger drainLog =
-      LoggerFactory.getLogger("org.apache.accumulo.manager.ManagerDrainImpl");
   private final Manager manager;
 
   protected ManagerClientServiceHandler(Manager manager) {
@@ -177,7 +161,7 @@
         }
       }
 
-      if (tableId.equals(RootTable.ID)) {
+      if (tableId.equals(AccumuloTable.ROOT.tableId())) {
         break; // this code does not properly handle the root tablet. See #798
       }
 
@@ -220,8 +204,8 @@
         }
 
       } catch (TabletDeletedException e) {
-        Manager.log.debug("Failed to scan {} table to wait for flush {}", MetadataTable.NAME,
-            tableId, e);
+        Manager.log.debug("Failed to scan {} table to wait for flush {}",
+            AccumuloTable.METADATA.tableName(), tableId, e);
       }
     }
 
@@ -394,9 +378,6 @@
               serverName, tablet);
         }
         break;
-      case CHOPPED:
-        manager.nextEvent.event("tablet %s chopped", tablet);
-        break;
     }
   }
 
@@ -634,107 +615,9 @@
     }
   }
 
-  @SuppressWarnings("deprecation")
-  @Override
-  public boolean drainReplicationTable(TInfo tfino, TCredentials credentials, String tableName,
-      Set<String> logsToWatch) throws TException {
-    AccumuloClient client = manager.getContext();
-
-    final Text tableId = new Text(getTableId(manager.getContext(), tableName).canonical());
-
-    drainLog.trace("Waiting for {} to be replicated for {}", logsToWatch, tableId);
-
-    drainLog.trace("Reading from metadata table");
-    final Set<Range> range = Collections.singleton(new Range(ReplicationSection.getRange()));
-    BatchScanner bs;
-    try {
-      bs = client.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    } catch (TableNotFoundException e) {
-      throw new RuntimeException("Could not read metadata table", e);
-    }
-    bs.setRanges(range);
-    bs.fetchColumnFamily(ReplicationSection.COLF);
-    try {
-      // Return immediately if there are records in metadata for these WALs
-      if (!allReferencesReplicated(bs, tableId, logsToWatch)) {
-        return false;
-      }
-    } finally {
-      bs.close();
-    }
-
-    drainLog.trace("reading from replication table");
-    try {
-      bs = client.createBatchScanner(org.apache.accumulo.core.replication.ReplicationTable.NAME,
-          Authorizations.EMPTY, 4);
-    } catch (TableNotFoundException e) {
-      throw new RuntimeException("Replication table was not found", e);
-    }
-    bs.setRanges(Collections.singleton(new Range()));
-    try {
-      // No records in metadata, check replication table
-      return allReferencesReplicated(bs, tableId, logsToWatch);
-    } finally {
-      bs.close();
-    }
-  }
-
   protected TableId getTableId(ClientContext context, String tableName)
       throws ThriftTableOperationException {
     return ClientServiceHandler.checkTableId(context, tableName, null);
   }
 
-  /**
-   * @return return true records are only in place which are fully replicated
-   */
-  @Deprecated
-  protected boolean allReferencesReplicated(BatchScanner bs, Text tableId,
-      Set<String> relevantLogs) {
-    Text rowHolder = new Text(), colfHolder = new Text();
-    for (Entry<Key,Value> entry : bs) {
-      drainLog.trace("Got key {}", entry.getKey().toStringNoTruncate());
-
-      entry.getKey().getColumnQualifier(rowHolder);
-      if (tableId.equals(rowHolder)) {
-        entry.getKey().getRow(rowHolder);
-        entry.getKey().getColumnFamily(colfHolder);
-
-        String file;
-        if (colfHolder.equals(ReplicationSection.COLF)) {
-          file = rowHolder.toString();
-          file = file.substring(ReplicationSection.getRowPrefix().length());
-        } else if (colfHolder
-            .equals(org.apache.accumulo.core.replication.ReplicationSchema.OrderSection.NAME)) {
-          file = org.apache.accumulo.core.replication.ReplicationSchema.OrderSection
-              .getFile(entry.getKey(), rowHolder);
-          long timeClosed = org.apache.accumulo.core.replication.ReplicationSchema.OrderSection
-              .getTimeClosed(entry.getKey(), rowHolder);
-          drainLog.trace("Order section: {} and {}", timeClosed, file);
-        } else {
-          file = rowHolder.toString();
-        }
-
-        // Skip files that we didn't observe when we started (new files/data)
-        if (relevantLogs.contains(file)) {
-          drainLog.trace("Found file that we *do* care about {}", file);
-        } else {
-          drainLog.trace("Found file that we didn't care about {}", file);
-          continue;
-        }
-
-        try {
-          Status stat = Status.parseFrom(entry.getValue().get());
-          if (!org.apache.accumulo.server.replication.StatusUtil.isFullyReplicated(stat)) {
-            drainLog.trace("{} and {} is not replicated", file, ProtobufUtil.toString(stat));
-            return false;
-          }
-          drainLog.trace("{} and {} is replicated", file, ProtobufUtil.toString(stat));
-        } catch (InvalidProtocolBufferException e) {
-          drainLog.trace("Could not parse protobuf for {}", entry.getKey(), e);
-        }
-      }
-    }
-
-    return true;
-  }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
index 8bd842f..fa7020a 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
@@ -20,11 +20,12 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -32,9 +33,13 @@
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
+import org.apache.accumulo.core.util.time.SteadyTime;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Keep a persistent roughly monotone view of how long a manager has been overseeing this cluster.
  */
@@ -47,9 +52,41 @@
 
   /**
    * Difference between time stored in ZooKeeper and System.nanoTime() when we last read from
-   * ZooKeeper.
+   * ZooKeeper. This offset may be negative or positive (depending on if the current nanoTime of the
+   * system is negative or positive) and is represented as a Duration to make computing future
+   * updates to the skewAmount and SteadyTime simpler.
+   * <p>
+   * Example where the skewAmount would be negative:
+   * <ul>
+   * <li>There's an existing persisted SteadyTime duration stored in Zookeeper from the total
+   * previous manager runs of 1,000,000</li>
+   * <li>Manager starts up and reads the previous value and the gets the current nano time which is
+   * 2,000,000</li>
+   * <li>The skew gets computed as the previous steady time duration minus the current time, so that
+   * becomes: 1,000,000 - 2,000,000 = -1,000,000 resulting in the skew value being negative 1
+   * million in this case</li>
+   * <li>When reading the current SteadyTime from the API, a new SteadyTime is computed by adding
+   * the current nano time plus the skew. So let's say 100,000 ns have elapsed since the start, so
+   * the current time is now 2,100,000. This results in:(-1,000,000) + 2,100,000 = 1,100,000. You
+   * end up with 1.1 million as a SteadyTime value that is the current elapsed time of 100,000 for
+   * the current manager run plus the previous SteadyTime of 1 million that was read on start.</li>
+   * </ul>
+   *
+   * Example where the skewAmount would be positive:
+   * <ul>
+   * <li>The current persisted value from previous runs is 1,000,000</li>
+   * <li>Manager starts up gets the current nano time which is -2,000,000</li>
+   * <li>The skew gets computed as: 1,000,000 - (-2,000,000) = 3,000,000 resulting in the skew value
+   * being positive 3 million in this case</li>
+   * <li>When reading the current SteadyTime from the API, a new SteadyTime is computed by adding
+   * the current nano time plus the skew. So let's say 100,000 ns have elapsed since the start, so
+   * the current time is now -1,900,000. This results in: (3,000,000) + (-1,900,000) = 1,100,000.
+   * You end up with 1.1 million as a SteadyTime value that is the current elapsed time of 100,000
+   * for the current manager run plus the previous SteadyTime of 1 million that was read on
+   * start.</li>
+   * </ul>
    */
-  private final AtomicLong skewAmount;
+  private final AtomicReference<Duration> skewAmount;
 
   public ManagerTime(Manager manager, AccumuloConfiguration conf) throws IOException {
     this.zPath = manager.getZooKeeperRoot() + Constants.ZMANAGER_TICK;
@@ -58,24 +95,23 @@
 
     try {
       zk.putPersistentData(zPath, "0".getBytes(UTF_8), NodeExistsPolicy.SKIP);
-      skewAmount =
-          new AtomicLong(Long.parseLong(new String(zk.getData(zPath), UTF_8)) - System.nanoTime());
+      skewAmount = new AtomicReference<>(updateSkew(getZkTime()));
     } catch (Exception ex) {
       throw new IOException("Error updating manager time", ex);
     }
 
     ThreadPools.watchCriticalScheduledTask(manager.getContext().getScheduledExecutor()
-        .scheduleWithFixedDelay(Threads.createNamedRunnable("Manager time keeper", () -> run()), 0,
+        .scheduleWithFixedDelay(Threads.createNamedRunnable("Manager time keeper", this::run), 0,
             SECONDS.toMillis(10), MILLISECONDS));
   }
 
   /**
    * How long has this cluster had a Manager?
    *
-   * @return Approximate total duration this cluster has had a Manager, in milliseconds.
+   * @return Approximate total duration this cluster has had a Manager
    */
-  public long getTime() {
-    return NANOSECONDS.toMillis(System.nanoTime() + skewAmount.get());
+  public SteadyTime getTime() {
+    return fromSkew(skewAmount.get());
   }
 
   public void run() {
@@ -86,8 +122,7 @@
       case INITIAL:
       case STOP:
         try {
-          long zkTime = Long.parseLong(new String(zk.getData(zPath), UTF_8));
-          skewAmount.set(zkTime - System.nanoTime());
+          skewAmount.set(updateSkew(getZkTime()));
         } catch (Exception ex) {
           if (log.isDebugEnabled()) {
             log.debug("Failed to retrieve manager tick time", ex);
@@ -101,8 +136,7 @@
       case UNLOAD_METADATA_TABLETS:
       case UNLOAD_ROOT_TABLET:
         try {
-          zk.putPersistentData(zPath,
-              Long.toString(System.nanoTime() + skewAmount.get()).getBytes(UTF_8),
+          zk.putPersistentData(zPath, serialize(fromSkew(skewAmount.get())),
               NodeExistsPolicy.OVERWRITE);
         } catch (Exception ex) {
           if (log.isDebugEnabled()) {
@@ -111,4 +145,62 @@
         }
     }
   }
+
+  private SteadyTime getZkTime() throws InterruptedException, KeeperException {
+    return deserialize(zk.getData(zPath));
+  }
+
+  /**
+   * Creates a new skewAmount from an existing SteadyTime steadyTime - System.nanoTime()
+   *
+   * @param steadyTime existing steadyTime
+   * @return Updated skew
+   */
+  @VisibleForTesting
+  static Duration updateSkew(SteadyTime steadyTime) {
+    return updateSkew(steadyTime, System.nanoTime());
+  }
+
+  /**
+   * Creates a new skewAmount from an existing SteadyTime by subtracting the given time value
+   *
+   * @param steadyTime existing steadyTime
+   * @param time time to subtract to update skew
+   * @return Updated skew
+   */
+  @VisibleForTesting
+  static Duration updateSkew(SteadyTime steadyTime, long time) {
+    return Duration.ofNanos(steadyTime.getNanos() - time);
+  }
+
+  /**
+   * Create a new SteadyTime from a skewAmount using System.nanoTime() + skewAmount
+   *
+   * @param skewAmount the skew amount to add
+   * @return A SteadyTime that has been skewed by the given skewAmount
+   */
+  @VisibleForTesting
+  static SteadyTime fromSkew(Duration skewAmount) {
+    return fromSkew(System.nanoTime(), skewAmount);
+  }
+
+  /**
+   * Create a new SteadyTime from a given time in ns and skewAmount using time + skewAmount
+   *
+   * @param time time to add the skew amount to
+   * @param skewAmount the skew amount to add
+   * @return A SteadyTime that has been skewed by the given skewAmount
+   */
+  @VisibleForTesting
+  static SteadyTime fromSkew(long time, Duration skewAmount) {
+    return SteadyTime.from(skewAmount.plusNanos(time));
+  }
+
+  static SteadyTime deserialize(byte[] steadyTime) {
+    return SteadyTime.from(Long.parseLong(new String(steadyTime, UTF_8)), TimeUnit.NANOSECONDS);
+  }
+
+  static byte[] serialize(SteadyTime steadyTime) {
+    return Long.toString(steadyTime.getNanos()).getBytes(UTF_8);
+  }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/MasterExecutable.java b/server/manager/src/main/java/org/apache/accumulo/manager/MasterExecutable.java
deleted file mode 100644
index 12ae0f0..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/MasterExecutable.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.manager;
-
-import org.apache.accumulo.start.spi.KeywordExecutable;
-
-import com.google.auto.service.AutoService;
-
-@Deprecated(since = "2.1.0")
-@AutoService(KeywordExecutable.class)
-public class MasterExecutable implements KeywordExecutable {
-
-  @Override
-  public String keyword() {
-    return "master";
-  }
-
-  @Override
-  public UsageGroup usageGroup() {
-    return UsageGroup.PROCESS;
-  }
-
-  @Override
-  public String description() {
-    return "Starts Accumulo master (Deprecated)";
-  }
-
-  @Override
-  public void execute(final String[] args) throws Exception {
-    Manager.main(args);
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
index 216526d..7adba90 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.manager;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.lang.Math.min;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -30,12 +30,16 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -55,9 +59,8 @@
 import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
@@ -65,19 +68,23 @@
 import org.apache.accumulo.core.metadata.TabletState;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 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.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataTime;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads.AccumuloDaemonThread;
 import org.apache.accumulo.manager.Manager.TabletGoalState;
 import org.apache.accumulo.manager.state.MergeStats;
@@ -101,9 +108,11 @@
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
 
 abstract class TabletGroupWatcher extends AccumuloDaemonThread {
   // Constants used to make sure assignment logging isn't excessive in quantity or size
@@ -250,9 +259,7 @@
               tls.getFutureServer(), tls.getCurrentServer(), tls.walogs.size());
 
           stats.update(tableId, state);
-          mergeStats.update(tls.extent, state, tls.chopped, !tls.walogs.isEmpty());
-          sendChopRequest(mergeStats.getMergeInfo(), state, tls);
-          sendSplitRequest(mergeStats.getMergeInfo(), state, tls);
+          mergeStats.update(tls.extent, state);
 
           // Always follow through with assignments
           if (state == TabletState.ASSIGNED) {
@@ -277,8 +284,8 @@
               switch (dependentLevel) {
                 case USER:
                   Set<TableId> onlineTables = manager.onlineTables();
-                  onlineTables.remove(RootTable.ID);
-                  onlineTables.remove(MetadataTable.ID);
+                  onlineTables.remove(AccumuloTable.ROOT.tableId());
+                  onlineTables.remove(AccumuloTable.METADATA.tableId());
                   userTablesExist = !onlineTables.isEmpty();
                   break;
                 case METADATA:
@@ -348,7 +355,7 @@
                     Manager.log.trace("[{}] Requesting TabletServer {} unload {} {}", store.name(),
                         location.getServerInstance(), tls.extent, goal.howUnload());
                     client.unloadTablet(manager.managerLock, tls.extent, goal.howUnload(),
-                        manager.getSteadyTime());
+                        manager.getSteadyTime().getMillis());
                     unloaded++;
                     totalUnloaded++;
                   } catch (TException tException) {
@@ -447,7 +454,7 @@
 
   private void hostSuspendedTablet(TabletLists tLists, TabletLocationState tls, Location location,
       TableConfiguration tableConf) {
-    if (manager.getSteadyTime() - tls.suspend.suspensionTime
+    if (manager.getSteadyTime().minus(tls.suspend.suspensionTime).toMillis()
         < tableConf.getTimeInMillis(Property.TABLE_SUSPEND_DURATION)) {
       // Tablet is suspended. See if its tablet server is back.
       TServerInstance returnInstance = null;
@@ -501,9 +508,9 @@
       Map<Key,Value> future = new HashMap<>();
       Map<Key,Value> assigned = new HashMap<>();
       KeyExtent extent = KeyExtent.fromMetaRow(row);
-      String table = MetadataTable.NAME;
+      String table = AccumuloTable.METADATA.tableName();
       if (extent.isMeta()) {
-        table = RootTable.NAME;
+        table = AccumuloTable.ROOT.tableName();
       }
       Scanner scanner = manager.getContext().createScanner(table, Authorizations.EMPTY);
       scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
@@ -562,83 +569,6 @@
     return result;
   }
 
-  private void sendSplitRequest(MergeInfo info, TabletState state, TabletLocationState tls) {
-    // Already split?
-    if (!info.getState().equals(MergeState.SPLITTING)) {
-      return;
-    }
-    // Merges don't split
-    if (!info.isDelete()) {
-      return;
-    }
-    // Online and ready to split?
-    if (!state.equals(TabletState.HOSTED)) {
-      return;
-    }
-    // Does this extent cover the end points of the delete?
-    KeyExtent range = info.getExtent();
-    if (tls.extent.overlaps(range)) {
-      for (Text splitPoint : new Text[] {range.prevEndRow(), range.endRow()}) {
-        if (splitPoint == null) {
-          continue;
-        }
-        if (!tls.extent.contains(splitPoint)) {
-          continue;
-        }
-        if (splitPoint.equals(tls.extent.endRow())) {
-          continue;
-        }
-        if (splitPoint.equals(tls.extent.prevEndRow())) {
-          continue;
-        }
-        try {
-          TServerConnection conn;
-          conn = manager.tserverSet.getConnection(tls.getCurrentServer());
-          if (conn != null) {
-            Manager.log.info("Asking {} to split {} at {}", tls.current, tls.extent, splitPoint);
-            conn.splitTablet(tls.extent, splitPoint);
-          } else {
-            Manager.log.warn("Not connected to server {}", tls.current);
-          }
-        } catch (NotServingTabletException e) {
-          Manager.log.debug("Error asking tablet server to split a tablet: ", e);
-        } catch (Exception e) {
-          Manager.log.warn("Error asking tablet server to split a tablet: ", e);
-        }
-      }
-    }
-  }
-
-  private void sendChopRequest(MergeInfo info, TabletState state, TabletLocationState tls) {
-    // Don't bother if we're in the wrong state
-    if (!info.getState().equals(MergeState.WAITING_FOR_CHOPPED)) {
-      return;
-    }
-    // Tablet must be online
-    if (!state.equals(TabletState.HOSTED)) {
-      return;
-    }
-    // Tablet isn't already chopped
-    if (tls.chopped) {
-      return;
-    }
-    // Tablet ranges intersect
-    if (info.needsToBeChopped(tls.extent)) {
-      TServerConnection conn;
-      try {
-        conn = manager.tserverSet.getConnection(tls.getCurrentServer());
-        if (conn != null) {
-          Manager.log.info("Asking {} to chop {}", tls.current, tls.extent);
-          conn.chop(manager.managerLock, tls.extent);
-        } else {
-          Manager.log.warn("Could not connect to server {}", tls.current);
-        }
-      } catch (TException e) {
-        Manager.log.warn("Communications error asking tablet server to chop a tablet");
-      }
-    }
-  }
-
   private void updateMergeState(Map<TableId,MergeStats> mergeStatsCache) {
     for (MergeStats stats : mergeStatsCache.values()) {
       try {
@@ -658,15 +588,28 @@
           try {
             if (stats.getMergeInfo().isDelete()) {
               deleteTablets(stats.getMergeInfo());
+              // For delete we are done and can skip to COMPLETE
+              update = MergeState.COMPLETE;
             } else {
               mergeMetadataRecords(stats.getMergeInfo());
+              // For merge we need another state to delete the tablets
+              // and clear the marker
+              update = MergeState.MERGED;
             }
-            update = MergeState.COMPLETE;
             manager.setMergeState(stats.getMergeInfo(), update);
           } catch (Exception ex) {
             Manager.log.error("Unable merge metadata table records", ex);
           }
         }
+
+        // If the state is MERGED then we are finished with metadata updates
+        if (update == MergeState.MERGED) {
+          // Finish the merge operatoin by deleting the merged tablets and
+          // cleaning up the marker that was used for merge
+          deleteMergedTablets(stats.getMergeInfo());
+          update = MergeState.COMPLETE;
+          manager.setMergeState(stats.getMergeInfo(), update);
+        }
       } catch (Exception ex) {
         Manager.log.error(
             "Unable to update merge state for merge " + stats.getMergeInfo().getExtent(), ex);
@@ -674,15 +617,137 @@
     }
   }
 
+  // Remove the merged marker from the last tablet in the merge range
+  private void clearMerged(MergeInfo mergeInfo, BatchWriter bw, HighTablet highTablet)
+      throws AccumuloException {
+    Manager.log.debug("Clearing MERGED marker for {}", mergeInfo.getExtent());
+    var m = new Mutation(highTablet.getExtent().toMetaRow());
+    MergedColumnFamily.MERGED_COLUMN.putDelete(m);
+    bw.addMutation(m);
+    bw.flush();
+  }
+
+  // This method finds returns the deletion starting row (exclusive) for tablets that
+  // need to be actually deleted. If the startTablet is null then
+  // the deletion start row will just be null as all tablets are being deleted
+  // up to the end. Otherwise, this returns the endRow of the first tablet
+  // as the first tablet should be kept and will have been previously
+  // fenced if necessary
+  private Text getDeletionStartRow(final KeyExtent startTablet) {
+    if (startTablet == null) {
+      Manager.log.debug("First tablet for delete range is null");
+      return null;
+    }
+
+    final Text deletionStartRow = startTablet.endRow();
+    Manager.log.debug("Start row is {} for deletion", deletionStartRow);
+
+    return deletionStartRow;
+  }
+
+  // This method finds returns the deletion ending row (inclusive) for tablets that
+  // need to be actually deleted. If the endTablet is null then
+  // the deletion end row will just be null as all tablets are being deleted
+  // after the start row. Otherwise, this returns the prevEndRow of the last tablet
+  // as the last tablet should be kept and will have been previously
+  // fenced if necessary
+  private Text getDeletionEndRow(final KeyExtent endTablet) {
+    if (endTablet == null) {
+      Manager.log.debug("Last tablet for delete range is null");
+      return null;
+    }
+
+    Text deletionEndRow = endTablet.prevEndRow();
+    Manager.log.debug("Deletion end row is {}", deletionEndRow);
+
+    return deletionEndRow;
+  }
+
+  private static boolean isFirstTabletInTable(KeyExtent tablet) {
+    return tablet != null && tablet.prevEndRow() == null;
+  }
+
+  private static boolean isLastTabletInTable(KeyExtent tablet) {
+    return tablet != null && tablet.endRow() == null;
+  }
+
+  private static boolean areContiguousTablets(KeyExtent firstTablet, KeyExtent lastTablet) {
+    return firstTablet != null && lastTablet != null
+        && Objects.equals(firstTablet.endRow(), lastTablet.prevEndRow());
+  }
+
+  private boolean hasTabletsToDelete(final KeyExtent firstTabletInRange,
+      final KeyExtent lastTableInRange) {
+    // If the tablets are equal (and not null) then the deletion range is just part of 1 tablet
+    // which will be fenced so there are no tablets to delete. The null check is because if both
+    // are null then we are just deleting everything, so we do have tablets to delete
+    if (Objects.equals(firstTabletInRange, lastTableInRange) && firstTabletInRange != null) {
+      Manager.log.trace(
+          "No tablets to delete, firstTablet {} equals lastTablet {} in deletion range and was fenced.",
+          firstTabletInRange, lastTableInRange);
+      return false;
+      // If the lastTablet of the deletion range is the first tablet of the table it has been fenced
+      // already so nothing to actually delete before it
+    } else if (isFirstTabletInTable(lastTableInRange)) {
+      Manager.log.trace(
+          "No tablets to delete, lastTablet {} in deletion range is the first tablet of the table and was fenced.",
+          lastTableInRange);
+      return false;
+      // If the firstTablet of the deletion range is the last tablet of the table it has been fenced
+      // already so nothing to actually delete after it
+    } else if (isLastTabletInTable(firstTabletInRange)) {
+      Manager.log.trace(
+          "No tablets to delete, firstTablet {} in deletion range is the last tablet of the table and was fenced.",
+          firstTabletInRange);
+      return false;
+      // If the firstTablet and lastTablet are contiguous tablets then there is nothing to delete as
+      // each will be fenced and nothing between
+    } else if (areContiguousTablets(firstTabletInRange, lastTableInRange)) {
+      Manager.log.trace(
+          "No tablets to delete, firstTablet {} and lastTablet {} in deletion range are contiguous and were fenced.",
+          firstTabletInRange, lastTableInRange);
+      return false;
+    }
+
+    return true;
+  }
+
   private void deleteTablets(MergeInfo info) throws AccumuloException {
-    KeyExtent extent = info.getExtent();
-    String targetSystemTable = extent.isMeta() ? RootTable.NAME : MetadataTable.NAME;
+    // Before updated metadata and get the first and last tablets which
+    // are fenced if necessary
+    final Pair<KeyExtent,KeyExtent> firstAndLastTablets = updateMetadataRecordsForDelete(info);
+
+    // Find the deletion start row (exclusive) for tablets that need to be actually deleted
+    // This will be null if deleting everything up until the end row or it will be
+    // the endRow of the first tablet as the first tablet should be kept and will have
+    // already been fenced if necessary
+    final Text deletionStartRow = getDeletionStartRow(firstAndLastTablets.getFirst());
+
+    // Find the deletion end row (inclusive) for tablets that need to be actually deleted
+    // This will be null if deleting everything after the starting row or it will be
+    // the prevEndRow of the last tablet as the last tablet should be kept and will have
+    // already been fenced if necessary
+    Text deletionEndRow = getDeletionEndRow(firstAndLastTablets.getSecond());
+
+    // check if there are any tablets to delete and if not return
+    if (!hasTabletsToDelete(firstAndLastTablets.getFirst(), firstAndLastTablets.getSecond())) {
+      Manager.log.trace("No tablets to delete for range {}, returning", info.getExtent());
+      return;
+    }
+
+    // Build an extent for the actual deletion range
+    final KeyExtent extent =
+        new KeyExtent(info.getExtent().tableId(), deletionEndRow, deletionStartRow);
+    Manager.log.debug("Tablet deletion range is {}", extent);
+    String targetSystemTable =
+        extent.isMeta() ? AccumuloTable.ROOT.tableName() : AccumuloTable.METADATA.tableName();
     Manager.log.debug("Deleting tablets for {}", extent);
     KeyExtent followingTablet = null;
     if (extent.endRow() != null) {
       Key nextExtent = new Key(extent.endRow()).followingKey(PartialKey.ROW);
       followingTablet =
-          getHighTablet(new KeyExtent(extent.tableId(), nextExtent.getRow(), extent.endRow()));
+          getHighTablet(new KeyExtent(extent.tableId(), nextExtent.getRow(), extent.endRow()))
+              .getExtent();
       Manager.log.debug("Found following tablet {}", followingTablet);
     }
     try {
@@ -709,7 +774,7 @@
         Key key = entry.getKey();
         if (key.compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
           var stf = new StoredTabletFile(key.getColumnQualifierData().toString());
-          datafilesAndDirs.add(ReferenceFile.forFile(stf.getTableId(), stf.getMetaUpdateDelete()));
+          datafilesAndDirs.add(ReferenceFile.forFile(stf.getTableId(), stf));
           if (datafilesAndDirs.size() > 1000) {
             ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs);
             datafilesAndDirs.clear();
@@ -760,7 +825,6 @@
         Mutation m = new Mutation(goalTablet.toMetaRow());
         TabletColumnFamily.PREV_ROW_COLUMN.put(m,
             TabletColumnFamily.encodePrevEndRow(extent.prevEndRow()));
-        ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
         bw.addMutation(m);
         bw.flush();
       } finally {
@@ -774,7 +838,8 @@
   private void mergeMetadataRecords(MergeInfo info) throws AccumuloException {
     KeyExtent range = info.getExtent();
     Manager.log.debug("Merging metadata for {}", range);
-    KeyExtent stop = getHighTablet(range);
+    HighTablet highTablet = getHighTablet(range);
+    KeyExtent stop = highTablet.getExtent();
     Manager.log.debug("Highest tablet is {}", stop);
     Value firstPrevRowValue = null;
     Text stopRow = stop.toMetaRow();
@@ -782,20 +847,32 @@
     if (start == null) {
       start = new Text();
     }
-    Range scanRange =
-        new Range(TabletsSection.encodeRow(range.tableId(), start), false, stopRow, false);
-    String targetSystemTable = MetadataTable.NAME;
+
+    String targetSystemTable = AccumuloTable.METADATA.tableName();
     if (range.isMeta()) {
-      targetSystemTable = RootTable.NAME;
+      targetSystemTable = AccumuloTable.ROOT.tableName();
     }
 
     AccumuloClient client = manager.getContext();
 
+    KeyExtent stopExtent = KeyExtent.fromMetaRow(stop.toMetaRow());
+    KeyExtent previousKeyExtent = null;
+    KeyExtent lastExtent = null;
+
+    // Check if we have already previously fenced the tablets
+    if (highTablet.isMerged()) {
+      Manager.log.debug("tablet metadata already fenced for merge {}", range);
+      // Return as we already fenced the files
+      return;
+    }
+
     try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) {
       long fileCount = 0;
       // Make file entries in highest tablet
       Scanner scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY);
-      scanner.setRange(scanRange);
+      // Update to set the range to include the highest tablet
+      scanner.setRange(
+          new Range(TabletsSection.encodeRow(range.tableId(), start), false, stopRow, true));
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       ServerColumnFamily.TIME_COLUMN.fetch(scanner);
       ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner);
@@ -817,8 +894,67 @@
           // Verify that Tablet has no WALs
         } else if (key.getColumnFamily().equals(LogColumnFamily.NAME)) {
           throw new IllegalStateException("Tablet " + key.getRow() + " has walogs during a merge!");
-        } else if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-          m.put(key.getColumnFamily(), key.getColumnQualifier(), value);
+        }
+
+        final KeyExtent keyExtent = KeyExtent.fromMetaRow(key.getRow());
+
+        // Keep track of the last Key Extent seen so we can use it to fence
+        // of RFiles when merging the metadata
+        if (lastExtent != null && !keyExtent.equals(lastExtent)) {
+          previousKeyExtent = lastExtent;
+        }
+
+        // Special case to handle the highest/stop tablet, which is where files are
+        // merged to. The existing merge code won't delete files from this tablet
+        // so we need to handle the deletes in this tablet when fencing files.
+        // We may be able to make this simpler in the future.
+        if (keyExtent.equals(stopExtent)) {
+          if (previousKeyExtent != null
+              && key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
+
+            // Fence off existing files by the end row of the previous tablet and current tablet
+            final StoredTabletFile existing = StoredTabletFile.of(key.getColumnQualifier());
+            // The end row should be inclusive for the current tablet and the previous end row
+            // should be exclusive for the start row
+            Range fenced = new Range(previousKeyExtent.endRow(), false, keyExtent.endRow(), true);
+
+            // Clip range if exists
+            fenced = existing.hasRange() ? existing.getRange().clip(fenced) : fenced;
+
+            final StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced);
+            // If the existing metadata does not match then we need to delete the old
+            // and replace with a new range
+            if (!existing.equals(newFile)) {
+              m.putDelete(DataFileColumnFamily.NAME, existing.getMetadataText());
+              m.put(key.getColumnFamily(), newFile.getMetadataText(), value);
+            }
+
+            fileCount++;
+          }
+          // For the highest tablet we only care about the DataFileColumnFamily
+          continue;
+        }
+
+        // Handle metadata updates for all other tablets except the highest tablet
+        // Ranges are created for the files and then added to the highest tablet in
+        // the merge range. Deletes are handled later for the old files when the tablets
+        // are removed.
+        if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
+          final StoredTabletFile existing = StoredTabletFile.of(key.getColumnQualifier());
+
+          // Fence off files by the previous tablet and current tablet that is being merged
+          // The end row should be inclusive for the current tablet and the previous end row should
+          // be exclusive for the start row.
+          Range fenced = new Range(previousKeyExtent != null ? previousKeyExtent.endRow() : null,
+              false, keyExtent.endRow(), true);
+
+          // Clip range with the tablet range if the range already exists
+          fenced = existing.hasRange() ? existing.getRange().clip(fenced) : fenced;
+
+          // Move the file and range to the last tablet
+          StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced);
+          m.put(key.getColumnFamily(), newFile.getMetadataText(), value);
+
           fileCount++;
         } else if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)
             && firstPrevRowValue == null) {
@@ -831,6 +967,8 @@
           var allVolumesDir = new AllVolumesDirectory(range.tableId(), value.toString());
           bw.addMutation(manager.getContext().getAmple().createDeleteMutation(allVolumesDir));
         }
+
+        lastExtent = keyExtent;
       }
 
       // read the logical time from the last tablet in the merge range, it is not included in
@@ -856,39 +994,284 @@
       // delete any entries for external compactions
       extCompIds.forEach(ecid -> m.putDelete(ExternalCompactionColumnFamily.STR_NAME, ecid));
 
-      if (!m.getUpdates().isEmpty()) {
-        bw.addMutation(m);
-      }
+      // Add a marker so we know the tablets have been fenced in case the merge operation
+      // needs to be recovered and restarted to finish later.
+      MergedColumnFamily.MERGED_COLUMN.put(m, MergedColumnFamily.MERGED_VALUE);
 
+      // Add the prev row column update to the same mutation as the
+      // file updates so it will be atomic and only update the prev row
+      // if the tablets were fenced
+      Preconditions.checkState(firstPrevRowValue != null,
+          "Previous row entry for lowest tablet was not found.");
+      stop = new KeyExtent(stop.tableId(), stop.endRow(),
+          TabletColumnFamily.decodePrevEndRow(firstPrevRowValue));
+      TabletColumnFamily.PREV_ROW_COLUMN.put(m,
+          TabletColumnFamily.encodePrevEndRow(stop.prevEndRow()));
+      Manager.log.debug("Setting the prevRow for last tablet: {}", stop);
+      bw.addMutation(m);
       bw.flush();
 
       Manager.log.debug("Moved {} files to {}", fileCount, stop);
-
-      if (firstPrevRowValue == null) {
-        Manager.log.debug("tablet already merged");
-        return;
-      }
-
-      stop = new KeyExtent(stop.tableId(), stop.endRow(),
-          TabletColumnFamily.decodePrevEndRow(firstPrevRowValue));
-      Mutation updatePrevRow = TabletColumnFamily.createPrevRowMutation(stop);
-      Manager.log.debug("Setting the prevRow for last tablet: {}", stop);
-      bw.addMutation(updatePrevRow);
-      bw.flush();
-
-      deleteTablets(info, scanRange, bw, client);
-
-      // Clean-up the last chopped marker
-      var m2 = new Mutation(stopRow);
-      ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m2);
-      bw.addMutation(m2);
-      bw.flush();
-
     } catch (Exception ex) {
       throw new AccumuloException(ex);
     }
   }
 
+  private void deleteMergedTablets(MergeInfo info) throws AccumuloException {
+    KeyExtent range = info.getExtent();
+    Manager.log.debug("Deleting merged tablets for {}", range);
+    HighTablet highTablet = getHighTablet(range);
+    if (!highTablet.isMerged()) {
+      Manager.log.debug("Tablets have already been deleted for merge with range {}, returning",
+          range);
+      return;
+    }
+
+    KeyExtent stop = highTablet.getExtent();
+    Manager.log.debug("Highest tablet is {}", stop);
+
+    Text stopRow = stop.toMetaRow();
+    Text start = range.prevEndRow();
+    if (start == null) {
+      start = new Text();
+    }
+    Range scanRange =
+        new Range(TabletsSection.encodeRow(range.tableId(), start), false, stopRow, false);
+    String targetSystemTable = AccumuloTable.METADATA.tableName();
+    if (range.isMeta()) {
+      targetSystemTable = AccumuloTable.ROOT.tableName();
+    }
+
+    AccumuloClient client = manager.getContext();
+
+    try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) {
+      // Continue and delete the tablets that were merged
+      deleteTablets(info, scanRange, bw, client);
+
+      // Clear the merged marker after we finish deleting tablets
+      clearMerged(info, bw, highTablet);
+    } catch (Exception ex) {
+      throw new AccumuloException(ex);
+    }
+  }
+
+  // This method is used to detect if a tablet needs to be split/chopped for a delete
+  // Instead of performing a split or chop compaction, the tablet will have its files fenced.
+  private boolean needsFencingForDeletion(MergeInfo info, KeyExtent keyExtent) {
+    // Does this extent cover the end points of the delete?
+    final Predicate<Text> isWithin = r -> r != null && keyExtent.contains(r);
+    final Predicate<Text> isNotBoundary =
+        r -> !r.equals(keyExtent.endRow()) && !r.equals(keyExtent.prevEndRow());
+    final KeyExtent deleteRange = info.getExtent();
+
+    return (keyExtent.overlaps(deleteRange) && Stream
+        .of(deleteRange.prevEndRow(), deleteRange.endRow()).anyMatch(isWithin.and(isNotBoundary)))
+        || info.needsToBeChopped(keyExtent);
+  }
+
+  // Instead of splitting or chopping tablets for a delete we instead create ranges
+  // to exclude the portion of the tablet that should be deleted
+  private Text followingRow(Text row) {
+    if (row == null) {
+      return null;
+    }
+    return new Key(row).followingKey(PartialKey.ROW).getRow();
+  }
+
+  // Instead of splitting or chopping tablets for a delete we instead create ranges
+  // to exclude the portion of the tablet that should be deleted
+  private List<Range> createRangesForDeletion(TabletMetadata tabletMetadata,
+      final KeyExtent deleteRange) {
+    final KeyExtent tabletExtent = tabletMetadata.getExtent();
+
+    // If the delete range wholly contains the tablet being deleted then there is no range to clip
+    // files to because the files should be completely dropped.
+    Preconditions.checkArgument(!deleteRange.contains(tabletExtent), "delete range:%s tablet:%s",
+        deleteRange, tabletExtent);
+
+    final List<Range> ranges = new ArrayList<>();
+
+    if (deleteRange.overlaps(tabletExtent)) {
+      if (deleteRange.prevEndRow() != null
+          && tabletExtent.contains(followingRow(deleteRange.prevEndRow()))) {
+        Manager.log.trace("Fencing tablet {} files to ({},{}]", tabletExtent,
+            tabletExtent.prevEndRow(), deleteRange.prevEndRow());
+        ranges.add(new Range(tabletExtent.prevEndRow(), false, deleteRange.prevEndRow(), true));
+      }
+
+      // This covers the case of when a deletion range overlaps the last tablet. We need to create a
+      // range that excludes the deletion.
+      if (deleteRange.endRow() != null
+          && tabletMetadata.getExtent().contains(deleteRange.endRow())) {
+        Manager.log.trace("Fencing tablet {} files to ({},{}]", tabletExtent, deleteRange.endRow(),
+            tabletExtent.endRow());
+        ranges.add(new Range(deleteRange.endRow(), false, tabletExtent.endRow(), true));
+      }
+    } else {
+      Manager.log.trace(
+          "Fencing tablet {} files to itself because it does not overlap delete range",
+          tabletExtent);
+      ranges.add(tabletExtent.toDataRange());
+    }
+
+    return ranges;
+  }
+
+  private Pair<KeyExtent,KeyExtent> updateMetadataRecordsForDelete(MergeInfo info)
+      throws AccumuloException {
+    final KeyExtent range = info.getExtent();
+
+    String targetSystemTable = AccumuloTable.METADATA.tableName();
+    if (range.isMeta()) {
+      targetSystemTable = AccumuloTable.ROOT.tableName();
+    }
+    final Pair<KeyExtent,KeyExtent> startAndEndTablets;
+
+    final AccumuloClient client = manager.getContext();
+
+    try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) {
+      final Text startRow = range.prevEndRow();
+      final Text endRow = range.endRow() != null
+          ? new Key(range.endRow()).followingKey(PartialKey.ROW).getRow() : null;
+
+      // Find the tablets that overlap the start and end row of the deletion range
+      // If the startRow is null then there will be an empty startTablet we don't need
+      // to fence a starting tablet as we are deleting everything up to the end tablet
+      // Likewise, if the endRow is null there will be an empty endTablet as we are deleting
+      // all tablets after the starting tablet
+      final Optional<TabletMetadata> startTablet = Optional.ofNullable(startRow).flatMap(
+          row -> loadTabletMetadata(range.tableId(), row, ColumnType.PREV_ROW, ColumnType.FILES));
+      final Optional<TabletMetadata> endTablet = Optional.ofNullable(endRow).flatMap(
+          row -> loadTabletMetadata(range.tableId(), row, ColumnType.PREV_ROW, ColumnType.FILES));
+
+      // Store the tablets in a Map if present so that if we have the same Tablet we
+      // only need to process the same tablet once when fencing
+      final SortedMap<KeyExtent,TabletMetadata> tabletMetadatas = new TreeMap<>();
+      startTablet.ifPresent(ft -> tabletMetadatas.put(ft.getExtent(), ft));
+      endTablet.ifPresent(lt -> tabletMetadatas.putIfAbsent(lt.getExtent(), lt));
+
+      // Capture the tablets to return them or null if not loaded
+      startAndEndTablets = new Pair<>(startTablet.map(TabletMetadata::getExtent).orElse(null),
+          endTablet.map(TabletMetadata::getExtent).orElse(null));
+
+      for (TabletMetadata tabletMetadata : tabletMetadatas.values()) {
+        final KeyExtent keyExtent = tabletMetadata.getExtent();
+
+        // Check if this tablet needs to have its files fenced for the deletion
+        if (needsFencingForDeletion(info, keyExtent)) {
+          Manager.log.debug("Found overlapping keyExtent {} for delete, fencing files.", keyExtent);
+
+          // Create the ranges for fencing the files, this takes the place of
+          // chop compactions and splits
+          final List<Range> ranges = createRangesForDeletion(tabletMetadata, range);
+          Preconditions.checkState(!ranges.isEmpty(),
+              "No ranges found that overlap deletion range.");
+
+          // Go through and fence each of the files that are part of the tablet
+          for (Entry<StoredTabletFile,DataFileValue> entry : tabletMetadata.getFilesMap()
+              .entrySet()) {
+            final StoredTabletFile existing = entry.getKey();
+            final DataFileValue value = entry.getValue();
+
+            final Mutation m = new Mutation(keyExtent.toMetaRow());
+
+            // Go through each range that was created and modify the metadata for the file
+            // The end row should be inclusive for the current tablet and the previous end row
+            // should be exclusive for the start row.
+            final Set<StoredTabletFile> newFiles = new HashSet<>();
+            final Set<StoredTabletFile> existingFile = Set.of(existing);
+
+            for (Range fenced : ranges) {
+              // Clip range with the tablet range if the range already exists
+              fenced = existing.hasRange() ? existing.getRange().clip(fenced, true) : fenced;
+
+              // If null the range is disjoint which can happen if there are existing fenced files
+              // If the existing file is disjoint then later we will delete if the file is not part
+              // of the newFiles set which means it is disjoint with all ranges
+              if (fenced != null) {
+                final StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced);
+                Manager.log.trace("Adding new file {} with range {}", newFile.getMetadataPath(),
+                    newFile.getRange());
+
+                // Add the new file to the newFiles set, it will be added later if it doesn't match
+                // the existing file already. We still need to add to the set to be checked later
+                // even if it matches the existing file as later the deletion logic will check to
+                // see if the existing file is part of this set before deleting. This is done to
+                // make sure the existing file isn't deleted unless it is not needed/disjoint
+                // with all ranges.
+                newFiles.add(newFile);
+              } else {
+                Manager.log.trace("Found a disjoint file {} with  range {} on delete",
+                    existing.getMetadataPath(), existing.getRange());
+              }
+            }
+
+            // If the existingFile is not contained in the newFiles set then we can delete it
+            Sets.difference(existingFile, newFiles).forEach(
+                delete -> m.putDelete(DataFileColumnFamily.NAME, existing.getMetadataText()));
+
+            // Add any new files that don't match the existingFile
+            // As of now we will only have at most 2 files as up to 2 ranges are created
+            final List<StoredTabletFile> filesToAdd =
+                new ArrayList<>(Sets.difference(newFiles, existingFile));
+            Preconditions.checkArgument(filesToAdd.size() <= 2,
+                "There should only be at most 2 StoredTabletFiles after computing new ranges.");
+
+            // If more than 1 new file then re-calculate the num entries and size
+            if (filesToAdd.size() == 2) {
+              // This splits up the values in half and makes sure they total the original
+              // values
+              final Pair<DataFileValue,DataFileValue> newDfvs = computeNewDfv(value);
+              m.put(DataFileColumnFamily.NAME, filesToAdd.get(0).getMetadataText(),
+                  newDfvs.getFirst().encodeAsValue());
+              m.put(DataFileColumnFamily.NAME, filesToAdd.get(1).getMetadataText(),
+                  newDfvs.getSecond().encodeAsValue());
+            } else {
+              // Will be 0 or 1 files
+              filesToAdd.forEach(newFile -> m.put(DataFileColumnFamily.NAME,
+                  newFile.getMetadataText(), value.encodeAsValue()));
+            }
+
+            if (!m.getUpdates().isEmpty()) {
+              bw.addMutation(m);
+            }
+          }
+        } else {
+          Manager.log.debug(
+              "Skipping metadata update on file for keyExtent {} for delete as not overlapping on rows.",
+              keyExtent);
+        }
+      }
+
+      bw.flush();
+
+      return startAndEndTablets;
+    } catch (Exception ex) {
+      throw new AccumuloException(ex);
+    }
+  }
+
+  // Divide each new DFV in half and make sure the sum equals the original
+  @VisibleForTesting
+  protected static Pair<DataFileValue,DataFileValue> computeNewDfv(DataFileValue value) {
+    final DataFileValue file1Value = new DataFileValue(Math.max(1, value.getSize() / 2),
+        Math.max(1, value.getNumEntries() / 2), value.getTime());
+
+    final DataFileValue file2Value =
+        new DataFileValue(Math.max(1, value.getSize() - file1Value.getSize()),
+            Math.max(1, value.getNumEntries() - file1Value.getNumEntries()), value.getTime());
+
+    return new Pair<>(file1Value, file2Value);
+  }
+
+  private Optional<TabletMetadata> loadTabletMetadata(TableId tabletId, final Text row,
+      ColumnType... columns) {
+    try (TabletsMetadata tabletsMetadata = manager.getContext().getAmple().readTablets()
+        .forTable(tabletId).overlapping(row, true, row).fetch(columns).build()) {
+      return tabletsMetadata.stream().findFirst();
+    }
+  }
+
   private KeyExtent deleteTablets(MergeInfo info, Range scanRange, BatchWriter bw,
       AccumuloClient client) throws TableNotFoundException, AccumuloException {
     Scanner scanner;
@@ -897,8 +1280,9 @@
     // group all deletes into tablet into one mutation, this makes tablets
     // either disappear entirely or not all.. this is important for the case
     // where the process terminates in the loop below...
-    scanner = client.createScanner(info.getExtent().isMeta() ? RootTable.NAME : MetadataTable.NAME,
-        Authorizations.EMPTY);
+    Manager.log.debug("Inside delete tablets");
+    scanner = client.createScanner(info.getExtent().isMeta() ? AccumuloTable.ROOT.tableName()
+        : AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
     Manager.log.debug("Deleting range {}", scanRange);
     scanner.setRange(scanRange);
     RowIterator rowIter = new RowIterator(scanner);
@@ -943,24 +1327,48 @@
         || key.getColumnFamily().equals(FutureLocationColumnFamily.NAME);
   }
 
-  private KeyExtent getHighTablet(KeyExtent range) throws AccumuloException {
+  private HighTablet getHighTablet(KeyExtent range) throws AccumuloException {
     try {
       AccumuloClient client = manager.getContext();
-      Scanner scanner = client.createScanner(range.isMeta() ? RootTable.NAME : MetadataTable.NAME,
+      Scanner scanner = client.createScanner(
+          range.isMeta() ? AccumuloTable.ROOT.tableName() : AccumuloTable.METADATA.tableName(),
           Authorizations.EMPTY);
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+      MergedColumnFamily.MERGED_COLUMN.fetch(scanner);
       KeyExtent start = new KeyExtent(range.tableId(), range.endRow(), null);
       scanner.setRange(new Range(start.toMetaRow(), null));
       Iterator<Entry<Key,Value>> iterator = scanner.iterator();
       if (!iterator.hasNext()) {
         throw new AccumuloException("No last tablet for a merge " + range);
       }
-      Entry<Key,Value> entry = iterator.next();
-      KeyExtent highTablet = KeyExtent.fromMetaPrevRow(entry);
-      if (!highTablet.tableId().equals(range.tableId())) {
+
+      KeyExtent highTablet = null;
+      boolean merged = false;
+      Text firstRow = null;
+
+      while (iterator.hasNext()) {
+        Entry<Key,Value> entry = iterator.next();
+        if (firstRow == null) {
+          firstRow = entry.getKey().getRow();
+        }
+        if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(entry.getKey())) {
+          Preconditions.checkState(entry.getKey().getRow().equals(firstRow),
+              "Row " + entry.getKey().getRow() + " does not match first row seen " + firstRow);
+          highTablet = KeyExtent.fromMetaPrevRow(entry);
+          Manager.log.debug("found high tablet: {}", entry.getKey());
+          break;
+        } else if (MergedColumnFamily.MERGED_COLUMN.hasColumns(entry.getKey())) {
+          Preconditions.checkState(entry.getKey().getRow().equals(firstRow),
+              "Row " + entry.getKey().getRow() + " does not match first row seen " + firstRow);
+          Manager.log.debug("is merged true: {}", entry.getKey());
+          merged = true;
+        }
+      }
+
+      if (highTablet == null || !highTablet.tableId().equals(range.tableId())) {
         throw new AccumuloException("No last tablet for merge " + range + " " + highTablet);
       }
-      return highTablet;
+      return new HighTablet(highTablet, merged);
     } catch (Exception ex) {
       throw new AccumuloException("Unexpected failure finding the last tablet for a merge " + range,
           ex);
@@ -1066,4 +1474,22 @@
     }
   }
 
+  @VisibleForTesting
+  protected static class HighTablet {
+    private final KeyExtent extent;
+    private final boolean merged;
+
+    public HighTablet(KeyExtent extent, boolean merged) {
+      this.extent = Objects.requireNonNull(extent);
+      this.merged = merged;
+    }
+
+    public boolean isMerged() {
+      return merged;
+    }
+
+    public KeyExtent getExtent() {
+      return extent;
+    }
+  }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java
index f0502bc..02a30d4 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java
@@ -26,19 +26,12 @@
 import org.apache.accumulo.core.metrics.MetricsProducer;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.metrics.fate.FateMetrics;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class ManagerMetrics {
 
-  private final static Logger log = LoggerFactory.getLogger(ManagerMetrics.class);
-
-  public static List<MetricsProducer> getProducers(AccumuloConfiguration conf, Manager m) {
+  public static List<MetricsProducer> getProducers(AccumuloConfiguration conf, Manager manager) {
     ArrayList<MetricsProducer> producers = new ArrayList<>();
-    @SuppressWarnings("deprecation")
-    ReplicationMetrics replMetrics = new ReplicationMetrics(m);
-    producers.add(replMetrics);
-    producers.add(new FateMetrics(m.getContext(),
+    producers.add(new FateMetrics(manager.getContext(),
         conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL)));
     return producers;
   }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
deleted file mode 100644
index 931fb30..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
+++ /dev/null
@@ -1,170 +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.manager.metrics;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metrics.MetricsProducer;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.server.replication.ReplicationUtil;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.micrometer.core.instrument.MeterRegistry;
-import io.micrometer.core.instrument.Timer;
-
-@Deprecated
-public class ReplicationMetrics implements MetricsProducer {
-
-  private static final Logger log = LoggerFactory.getLogger(ReplicationMetrics.class);
-
-  private final Manager manager;
-  private final ReplicationUtil replicationUtil;
-  private final Map<Path,Long> pathModTimes;
-
-  private Timer replicationQueueTimer;
-  private AtomicLong pendingFiles;
-  private AtomicInteger numPeers;
-  private AtomicInteger maxReplicationThreads;
-
-  ReplicationMetrics(Manager manager) {
-    this.manager = manager;
-    pathModTimes = new HashMap<>();
-    replicationUtil = new ReplicationUtil(manager.getContext());
-  }
-
-  protected void update() {
-    // Only add these metrics if the replication table is online and there are peers
-    if (TableState.ONLINE == manager.getContext().getTableState(ReplicationTable.ID)
-        && !replicationUtil.getPeers().isEmpty()) {
-      pendingFiles.set(getNumFilesPendingReplication());
-      addReplicationQueueTimeMetrics();
-    } else {
-      pendingFiles.set(0);
-    }
-    numPeers.set(getNumConfiguredPeers());
-    maxReplicationThreads.set(getMaxReplicationThreads());
-  }
-
-  protected long getNumFilesPendingReplication() {
-    // The total set of configured targets
-    Set<ReplicationTarget> allConfiguredTargets = replicationUtil.getReplicationTargets();
-
-    // Number of files per target we have to replicate
-    Map<ReplicationTarget,Long> targetCounts = replicationUtil.getPendingReplications();
-
-    long filesPending = 0;
-
-    // Sum pending replication over all targets
-    for (ReplicationTarget configuredTarget : allConfiguredTargets) {
-      Long numFiles = targetCounts.get(configuredTarget);
-
-      if (numFiles != null) {
-        filesPending += numFiles;
-      }
-    }
-
-    return filesPending;
-  }
-
-  protected int getNumConfiguredPeers() {
-    return replicationUtil.getPeers().size();
-  }
-
-  protected int getMaxReplicationThreads() {
-    return replicationUtil.getMaxReplicationThreads(manager.getManagerMonitorInfo());
-  }
-
-  protected void addReplicationQueueTimeMetrics() {
-    Set<Path> paths = replicationUtil.getPendingReplicationPaths();
-
-    // We'll take a snap of the current time and use this as a diff between any deleted
-    // file's modification time and now. The reported latency will be off by at most a
-    // number of seconds equal to the metric polling period
-    long currentTime = System.currentTimeMillis();
-
-    // Iterate through all the pending paths and update the mod time if we don't know it yet
-    for (Path path : paths) {
-      if (!pathModTimes.containsKey(path)) {
-        try {
-          pathModTimes.put(path,
-              manager.getVolumeManager().getFileStatus(path).getModificationTime());
-        } catch (IOException e) {
-          // Ignore all IOExceptions
-          // Either the system is unavailable, or the file was deleted since the initial scan and
-          // this check
-          log.trace(
-              "Failed to get file status for {}, file system is unavailable or it does not exist",
-              path);
-        }
-      }
-    }
-
-    // Remove all currently pending files
-    Set<Path> deletedPaths = new HashSet<>(pathModTimes.keySet());
-    deletedPaths.removeAll(paths);
-
-    // Exit early if we have no replicated files to report on
-    if (deletedPaths.isEmpty()) {
-      return;
-    }
-
-    for (Path path : deletedPaths) {
-      // Remove this path and add the latency
-      Long modTime = pathModTimes.remove(path);
-      if (modTime != null) {
-        long diff = Math.max(0, currentTime - modTime);
-        // micrometer timer
-        replicationQueueTimer.record(Duration.ofMillis(diff));
-      }
-    }
-  }
-
-  @Override
-  public void registerMetrics(MeterRegistry registry) {
-    replicationQueueTimer = registry.timer(METRICS_REPLICATION_QUEUE);
-    pendingFiles = registry.gauge(METRICS_REPLICATION_PENDING_FILES, new AtomicLong(0));
-    numPeers = registry.gauge(METRICS_REPLICATION_PEERS, new AtomicInteger(0));
-    maxReplicationThreads = registry.gauge(METRICS_REPLICATION_THREADS, new AtomicInteger(0));
-
-    ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools()
-        .createScheduledExecutorService(1, "replicationMetricsPoller");
-    Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
-    long minimumRefreshDelay = TimeUnit.SECONDS.toMillis(5);
-    ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(this::update, minimumRefreshDelay,
-        minimumRefreshDelay, TimeUnit.MILLISECONDS);
-    ThreadPools.watchNonCriticalScheduledTask(future);
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
index f405881..cb590a3 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
@@ -22,13 +22,13 @@
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
@@ -38,9 +38,9 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.log.SortedLogState;
 import org.apache.accumulo.server.manager.recovery.HadoopLogCloser;
@@ -52,8 +52,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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 RecoveryManager {
 
@@ -70,7 +70,7 @@
   public RecoveryManager(Manager manager, long timeToCacheExistsInMillis) {
     this.manager = manager;
     existenceCache =
-        CacheBuilder.newBuilder().expireAfterWrite(timeToCacheExistsInMillis, TimeUnit.MILLISECONDS)
+        Caffeine.newBuilder().expireAfterWrite(timeToCacheExistsInMillis, TimeUnit.MILLISECONDS)
             .maximumWeight(10_000_000).weigher((path, exist) -> path.toString().length()).build();
 
     executor =
@@ -144,81 +144,80 @@
 
   private boolean exists(final Path path) throws IOException {
     try {
-      return existenceCache.get(path, () -> manager.getVolumeManager().exists(path));
-    } catch (ExecutionException e) {
+      return existenceCache.get(path, k -> {
+        try {
+          return manager.getVolumeManager().exists(path);
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      });
+    } catch (UncheckedIOException e) {
       throw new IOException(e);
     }
   }
 
-  public boolean recoverLogs(KeyExtent extent, Collection<Collection<String>> walogs)
-      throws IOException {
+  public boolean recoverLogs(KeyExtent extent, Collection<LogEntry> walogs) throws IOException {
     boolean recoveryNeeded = false;
 
-    for (Collection<String> logs : walogs) {
-      for (String walog : logs) {
+    for (LogEntry walog : walogs) {
 
-        Path switchedWalog = VolumeUtil.switchVolume(walog, FileType.WAL,
-            manager.getContext().getVolumeReplacements());
-        if (switchedWalog != null) {
-          // replaces the volume used for sorting, but do not change entry in metadata table. When
-          // the tablet loads it will change the metadata table entry. If
-          // the tablet has the same replacement config, then it will find the sorted log.
-          log.info("Volume replaced {} -> {}", walog, switchedWalog);
-          walog = switchedWalog.toString();
-        }
+      LogEntry switchedWalog =
+          VolumeUtil.switchVolume(walog, manager.getContext().getVolumeReplacements());
+      if (switchedWalog != null) {
+        // replaces the volume used for sorting, but do not change entry in metadata table. When
+        // the tablet loads it will change the metadata table entry. If
+        // the tablet has the same replacement config, then it will find the sorted log.
+        log.info("Volume replaced {} -> {}", walog, switchedWalog);
+        walog = switchedWalog;
+      }
 
-        String[] parts = walog.split("/");
-        String sortId = parts[parts.length - 1];
-        String filename = new Path(walog).toString();
-        String dest = RecoveryPath.getRecoveryPath(new Path(filename)).toString();
+      String sortId = walog.getUniqueID().toString();
+      String filename = walog.getPath();
+      String dest = RecoveryPath.getRecoveryPath(new Path(filename)).toString();
 
-        boolean sortQueued;
+      boolean sortQueued;
+      synchronized (this) {
+        sortQueued = sortsQueued.contains(sortId);
+      }
+
+      if (sortQueued
+          && zooCache.get(manager.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId)
+              == null) {
         synchronized (this) {
-          sortQueued = sortsQueued.contains(sortId);
+          sortsQueued.remove(sortId);
         }
+      }
 
-        if (sortQueued
-            && zooCache.get(manager.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId)
-                == null) {
-          synchronized (this) {
-            sortsQueued.remove(sortId);
-          }
-        }
-
-        if (exists(SortedLogState.getFinishedMarkerPath(dest))) {
-          synchronized (this) {
-            closeTasksQueued.remove(sortId);
-            recoveryDelay.remove(sortId);
-            sortsQueued.remove(sortId);
-          }
-          continue;
-        }
-
-        recoveryNeeded = true;
+      if (exists(SortedLogState.getFinishedMarkerPath(dest))) {
         synchronized (this) {
-          if (!closeTasksQueued.contains(sortId) && !sortsQueued.contains(sortId)) {
-            AccumuloConfiguration aconf = manager.getConfiguration();
-            @SuppressWarnings("deprecation")
-            LogCloser closer = Property.createInstanceFromPropertyName(aconf,
-                aconf.resolve(Property.MANAGER_WAL_CLOSER_IMPLEMENTATION,
-                    Property.MANAGER_WALOG_CLOSER_IMPLEMETATION),
-                LogCloser.class, new HadoopLogCloser());
-            Long delay = recoveryDelay.get(sortId);
-            if (delay == null) {
-              delay = aconf.getTimeInMillis(Property.MANAGER_RECOVERY_DELAY);
-            } else {
-              delay = Math.min(2 * delay, 1000 * 60 * 5L);
-            }
+          closeTasksQueued.remove(sortId);
+          recoveryDelay.remove(sortId);
+          sortsQueued.remove(sortId);
+        }
+        continue;
+      }
 
-            log.info("Starting recovery of {} (in : {}s), tablet {} holds a reference", filename,
-                (delay / 1000), extent);
-
-            ScheduledFuture<?> future = executor.schedule(
-                new LogSortTask(closer, filename, dest, sortId), delay, TimeUnit.MILLISECONDS);
-            ThreadPools.watchNonCriticalScheduledTask(future);
-            closeTasksQueued.add(sortId);
-            recoveryDelay.put(sortId, delay);
+      recoveryNeeded = true;
+      synchronized (this) {
+        if (!closeTasksQueued.contains(sortId) && !sortsQueued.contains(sortId)) {
+          AccumuloConfiguration aconf = manager.getConfiguration();
+          LogCloser closer = Property.createInstanceFromPropertyName(aconf,
+              Property.MANAGER_WAL_CLOSER_IMPLEMENTATION, LogCloser.class, new HadoopLogCloser());
+          Long delay = recoveryDelay.get(sortId);
+          if (delay == null) {
+            delay = aconf.getTimeInMillis(Property.MANAGER_RECOVERY_DELAY);
+          } else {
+            delay = Math.min(2 * delay, 1000 * 60 * 5L);
           }
+
+          log.info("Starting recovery of {} (in : {}s), tablet {} holds a reference", filename,
+              (delay / 1000), extent);
+
+          ScheduledFuture<?> future = executor.schedule(
+              new LogSortTask(closer, filename, dest, sortId), delay, TimeUnit.MILLISECONDS);
+          ThreadPools.watchNonCriticalScheduledTask(future);
+          closeTasksQueued.add(sortId);
+          recoveryDelay.put(sortId, delay);
         }
       }
     }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java
deleted file mode 100644
index 7e60c34..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssigner.java
+++ /dev/null
@@ -1,267 +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.manager.replication;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.util.Collection;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.Scanner;
-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.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.WorkAssigner;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Common methods for {@link WorkAssigner}s
- */
-@Deprecated
-public abstract class DistributedWorkQueueWorkAssigner implements WorkAssigner {
-  private static final Logger log = LoggerFactory.getLogger(DistributedWorkQueueWorkAssigner.class);
-
-  protected boolean isWorkRequired(Status status) {
-    return StatusUtil.isWorkRequired(status);
-  }
-
-  protected AccumuloClient client;
-  protected AccumuloConfiguration conf;
-  protected DistributedWorkQueue workQueue;
-  protected int maxQueueSize;
-  protected ZooCache zooCache;
-
-  protected void setClient(AccumuloClient client) {
-    this.client = client;
-  }
-
-  protected void setWorkQueue(DistributedWorkQueue workQueue) {
-    this.workQueue = workQueue;
-  }
-
-  protected void setMaxQueueSize(int maxQueueSize) {
-    this.maxQueueSize = maxQueueSize;
-  }
-
-  protected void setZooCache(ZooCache zooCache) {
-    this.zooCache = zooCache;
-  }
-
-  /**
-   * Initialize the DistributedWorkQueue using the proper ZK location
-   */
-  protected void initializeWorkQueue(AccumuloConfiguration conf) {
-    workQueue =
-        new DistributedWorkQueue(ZooUtil.getRoot(client.instanceOperations().getInstanceId())
-            + ReplicationConstants.ZOO_WORK_QUEUE, conf, this.workQueue.getContext());
-  }
-
-  @Override
-  public void configure(AccumuloConfiguration conf, AccumuloClient client) {
-    this.conf = conf;
-    this.client = client;
-  }
-
-  @Override
-  public void assignWork() {
-    if (workQueue == null) {
-      initializeWorkQueue(conf);
-    }
-
-    initializeQueuedWork();
-
-    if (zooCache == null) {
-      zooCache = new ZooCache(workQueue.getContext().getZooReader(), null);
-    }
-
-    // Get the maximum number of entries we want to queue work for (or the default)
-    this.maxQueueSize = conf.getCount(Property.REPLICATION_MAX_WORK_QUEUE);
-
-    // Scan over the work records, adding the work to the queue
-    createWork();
-
-    // Keep the state of the work we queued correct
-    cleanupFinishedWork();
-  }
-
-  /**
-   * Scan over the {@link WorkSection} of the replication table adding work for entries that have
-   * data to replicate and have not already been queued.
-   */
-  protected void createWork() {
-    // Create a scanner over the replication table's order entries
-    Scanner s;
-    try {
-      s = ReplicationTable.getScanner(client);
-    } catch (ReplicationTableOfflineException e) {
-      // no work to do; replication is off
-      return;
-    }
-
-    OrderSection.limit(s);
-
-    Text buffer = new Text();
-    for (Entry<Key,Value> orderEntry : s) {
-      // If we're not working off the entries, we need to not shoot ourselves in the foot by
-      // continuing
-      // to add more work entries
-      if (getQueueSize() > maxQueueSize) {
-        log.warn("Queued replication work exceeds configured maximum ({}),"
-            + " sleeping to allow work to occur", maxQueueSize);
-        return;
-      }
-
-      String file = OrderSection.getFile(orderEntry.getKey(), buffer);
-      OrderSection.getTableId(orderEntry.getKey(), buffer);
-      String sourceTableId = buffer.toString();
-
-      log.info("Determining if {} from {} needs to be replicated", file, sourceTableId);
-
-      Scanner workScanner;
-      try {
-        workScanner = ReplicationTable.getScanner(client);
-      } catch (ReplicationTableOfflineException e) {
-        log.warn("Replication table is offline. Will retry...");
-        sleepUninterruptibly(5, TimeUnit.SECONDS);
-        return;
-      }
-
-      WorkSection.limit(workScanner);
-      workScanner.setRange(Range.exact(file));
-
-      int newReplicationTasksSubmitted = 0, workEntriesRead = 0;
-      // For a file, we can concurrently replicate it to multiple targets
-      for (Entry<Key,Value> workEntry : workScanner) {
-        workEntriesRead++;
-        Status status;
-        try {
-          status = StatusUtil.fromValue(workEntry.getValue());
-        } catch (InvalidProtocolBufferException e) {
-          log.warn("Could not deserialize protobuf from work entry for {} to {}, will retry", file,
-              ReplicationTarget.from(workEntry.getKey().getColumnQualifier()), e);
-          continue;
-        }
-
-        // Get the ReplicationTarget for this Work record
-        ReplicationTarget target = WorkSection.getTarget(workEntry.getKey(), buffer);
-
-        // Get the file (if any) currently being replicated to the given peer for the given source
-        // table
-        Collection<String> keysBeingReplicated = getQueuedWork(target);
-
-        Path p = new Path(file);
-        String filename = p.getName();
-        String key = DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename, target);
-
-        if (!shouldQueueWork(target)) {
-          if (!isWorkRequired(status) && keysBeingReplicated.contains(key)) {
-            log.debug("Removing {} from replication state to {} because replication is complete",
-                key, target.getPeerName());
-            this.removeQueuedWork(target, key);
-          }
-
-          continue;
-        }
-
-        // If there is work to do
-        if (isWorkRequired(status)) {
-          if (queueWork(p, target)) {
-            newReplicationTasksSubmitted++;
-          }
-        } else {
-          log.debug("Not queueing work for {} to {} because {} doesn't need replication", file,
-              target, ProtobufUtil.toString(status));
-          if (keysBeingReplicated.contains(key)) {
-            log.debug("Removing {} from replication state to {} because replication is complete",
-                key, target.getPeerName());
-            this.removeQueuedWork(target, key);
-          }
-        }
-      }
-
-      log.debug("Read {} replication entries from the WorkSection of the replication table",
-          workEntriesRead);
-      log.info("Assigned {} replication work entries for {}", newReplicationTasksSubmitted, file);
-    }
-  }
-
-  /**
-   * @return Can replication work for the given {@link ReplicationTarget} be submitted to be worked
-   *         on.
-   */
-  protected abstract boolean shouldQueueWork(ReplicationTarget target);
-
-  /**
-   * @return the size of the queued work
-   */
-  protected abstract int getQueueSize();
-
-  /**
-   * Set up any internal state before using the WorkAssigner
-   */
-  protected abstract void initializeQueuedWork();
-
-  /**
-   * Queue the given work for the target
-   *
-   * @param path File to replicate
-   * @param target Target for the work
-   * @return True if the work was queued, false otherwise
-   */
-  protected abstract boolean queueWork(Path path, ReplicationTarget target);
-
-  /**
-   * @param target Target for the work
-   * @return Queued work for the given target
-   */
-  protected abstract Set<String> getQueuedWork(ReplicationTarget target);
-
-  /**
-   * Remove the given work from the internal state
-   */
-  protected abstract void removeQueuedWork(ReplicationTarget target, String queueKey);
-
-  /**
-   * Remove finished replication work from the internal state
-   */
-  protected abstract void cleanupFinishedWork();
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/FinishedWorkUpdater.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/FinishedWorkUpdater.java
deleted file mode 100644
index b533bb3..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/FinishedWorkUpdater.java
+++ /dev/null
@@ -1,190 +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.manager.replication;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-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.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Update the status record in the replication table with work that has been replicated to each
- * configured peer.
- */
-@Deprecated
-public class FinishedWorkUpdater implements Runnable {
-  private static final Logger log = LoggerFactory.getLogger(FinishedWorkUpdater.class);
-
-  private final AccumuloClient client;
-
-  public FinishedWorkUpdater(AccumuloClient client) {
-    this.client = client;
-  }
-
-  @Override
-  public void run() {
-    log.trace("Looking for finished replication work");
-
-    if (!ReplicationTable.isOnline(client)) {
-      log.trace("Replication table is not yet online, will retry");
-      return;
-    }
-
-    BatchScanner bs;
-    BatchWriter replBw;
-    try {
-      bs = ReplicationTable.getBatchScanner(client, 4);
-      replBw = ReplicationTable.getBatchWriter(client);
-    } catch (ReplicationTableOfflineException e) {
-      log.debug("Table is no longer online, will retry");
-      return;
-    }
-
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    bs.addScanIterator(cfg);
-    WorkSection.limit(bs);
-    bs.setRanges(Collections.singleton(new Range()));
-
-    try {
-      for (Entry<Key,Value> serializedRow : bs) {
-        SortedMap<Key,Value> wholeRow;
-        try {
-          wholeRow = WholeRowIterator.decodeRow(serializedRow.getKey(), serializedRow.getValue());
-        } catch (IOException e) {
-          log.warn("Could not deserialize whole row with key {}",
-              serializedRow.getKey().toStringNoTruncate(), e);
-          continue;
-        }
-
-        log.debug("Processing work progress for {} with {} columns",
-            serializedRow.getKey().getRow(), wholeRow.size());
-
-        Map<TableId,Long> tableIdToProgress = new HashMap<>();
-        boolean error = false;
-        Text buffer = new Text();
-
-        // We want to determine what the minimum point that all Work entries have replicated to
-        for (Entry<Key,Value> entry : wholeRow.entrySet()) {
-          Status status;
-          try {
-            status = Status.parseFrom(entry.getValue().get());
-          } catch (InvalidProtocolBufferException e) {
-            log.warn("Could not deserialize protobuf for {}", entry.getKey(), e);
-            error = true;
-            break;
-          }
-
-          // Get the replication target for the work record
-          entry.getKey().getColumnQualifier(buffer);
-          ReplicationTarget target = ReplicationTarget.from(buffer);
-
-          // Initialize the value in the map if we don't have one
-          if (!tableIdToProgress.containsKey(target.getSourceTableId())) {
-            tableIdToProgress.put(target.getSourceTableId(), Long.MAX_VALUE);
-          }
-
-          // Find the minimum value for begin (everyone has replicated up to this offset in the
-          // file)
-          tableIdToProgress.put(target.getSourceTableId(),
-              Math.min(tableIdToProgress.get(target.getSourceTableId()), status.getBegin()));
-        }
-
-        if (error) {
-          continue;
-        }
-
-        // Update the replication table for each source table we found work records for
-        for (Entry<TableId,Long> entry : tableIdToProgress.entrySet()) {
-          // If the progress is 0, then no one has replicated anything, and we don't need to update
-          // anything
-          if (entry.getValue() == 0) {
-            continue;
-          }
-
-          serializedRow.getKey().getRow(buffer);
-
-          log.debug("For {}, source table ID {} has replicated through {}",
-              serializedRow.getKey().getRow(), entry.getKey(), entry.getValue());
-
-          Mutation replMutation = new Mutation(buffer);
-
-          // Set that we replicated at least this much data, ignoring the other fields
-          Status updatedStatus = StatusUtil.replicated(entry.getValue());
-          Value serializedUpdatedStatus = ProtobufUtil.toValue(updatedStatus);
-
-          // Pull the sourceTableId into a Text
-          TableId srcTableId = entry.getKey();
-
-          // Make the mutation
-          StatusSection.add(replMutation, srcTableId, serializedUpdatedStatus);
-
-          log.debug("Updating replication status entry for {} with {}",
-              serializedRow.getKey().getRow(), ProtobufUtil.toString(updatedStatus));
-
-          try {
-            replBw.addMutation(replMutation);
-          } catch (MutationsRejectedException e) {
-            log.error("Error writing mutations to update replication Status"
-                + " messages in StatusSection, will retry", e);
-            return;
-          }
-        }
-      }
-    } finally {
-      log.debug("Finished updating files with completed replication work");
-
-      bs.close();
-
-      try {
-        replBw.close();
-      } catch (MutationsRejectedException e) {
-        log.error("Error writing mutations to update replication Status"
-            + " messages in StatusSection, will retry", e);
-      }
-    }
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java
deleted file mode 100644
index 9063ad7..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java
+++ /dev/null
@@ -1,110 +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.manager.replication;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.security.SecureRandom;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.fate.zookeeper.ZooReader;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinatorErrorCode;
-import org.apache.accumulo.core.replication.thrift.ReplicationCoordinatorException;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.server.security.SecurityOperation;
-import org.apache.thrift.TException;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Choose a tserver to service a replication task
- */
-@Deprecated
-public class ManagerReplicationCoordinator implements ReplicationCoordinator.Iface {
-  private static final SecureRandom random = new SecureRandom();
-  private static final Logger log = LoggerFactory.getLogger(ManagerReplicationCoordinator.class);
-
-  private final Manager manager;
-  private final ZooReader reader;
-  private final SecurityOperation security;
-
-  public ManagerReplicationCoordinator(Manager manager) {
-    this(manager, manager.getContext().getZooReader());
-  }
-
-  protected ManagerReplicationCoordinator(Manager manager, ZooReader reader) {
-    this.manager = manager;
-    this.reader = reader;
-    this.security = manager.getContext().getSecurityOperation();
-  }
-
-  @Override
-  public String getServicerAddress(String remoteTableId, TCredentials creds)
-      throws ReplicationCoordinatorException, TException {
-    try {
-      security.authenticateUser(manager.getContext().rpcCreds(), creds);
-    } catch (ThriftSecurityException e) {
-      log.error("{} failed to authenticate for replication to {}", creds.getPrincipal(),
-          remoteTableId);
-      throw new ReplicationCoordinatorException(ReplicationCoordinatorErrorCode.CANNOT_AUTHENTICATE,
-          "Could not authenticate " + creds.getPrincipal());
-    }
-
-    Set<TServerInstance> tservers = manager.onlineTabletServers();
-    if (tservers.isEmpty()) {
-      throw new ReplicationCoordinatorException(
-          ReplicationCoordinatorErrorCode.NO_AVAILABLE_SERVERS,
-          "No tservers are available for replication");
-    }
-
-    TServerInstance tserver = getRandomTServer(tservers, random.nextInt(tservers.size()));
-    String replServiceAddr;
-    try {
-      replServiceAddr = new String(reader.getData(manager.getZooKeeperRoot()
-          + ReplicationConstants.ZOO_TSERVERS + "/" + tserver.getHostPort()), UTF_8);
-    } catch (KeeperException | InterruptedException e) {
-      log.error("Could not fetch replication service port for tserver", e);
-      throw new ReplicationCoordinatorException(
-          ReplicationCoordinatorErrorCode.SERVICE_CONFIGURATION_UNAVAILABLE,
-          "Could not determine port for replication service running at " + tserver.getHostPort());
-    }
-
-    return replServiceAddr;
-  }
-
-  protected TServerInstance getRandomTServer(Set<TServerInstance> tservers, int offset) {
-    checkArgument(tservers.size() > offset, "Must provide an offset less than the size of the set");
-    Iterator<TServerInstance> iter = tservers.iterator();
-    while (offset > 0) {
-      iter.next();
-      offset--;
-    }
-
-    return iter.next();
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/RemoveCompleteReplicationRecords.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/RemoveCompleteReplicationRecords.java
deleted file mode 100644
index 76541d4..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/RemoveCompleteReplicationRecords.java
+++ /dev/null
@@ -1,225 +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.manager.replication;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.SortedMap;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-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.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Delete replication entries from the replication table that are fully replicated and closed
- */
-@Deprecated
-public class RemoveCompleteReplicationRecords implements Runnable {
-  private static final Logger log = LoggerFactory.getLogger(RemoveCompleteReplicationRecords.class);
-
-  private AccumuloClient client;
-
-  public RemoveCompleteReplicationRecords(AccumuloClient client) {
-    this.client = client;
-  }
-
-  @Override
-  public void run() {
-    BatchScanner bs;
-    BatchWriter bw;
-    try {
-      bs = ReplicationTable.getBatchScanner(client, 4);
-      bw = ReplicationTable.getBatchWriter(client);
-
-      if (bs == null || bw == null) {
-        throw new AssertionError("Inconceivable; an exception should have been"
-            + " thrown, but 'bs' or 'bw' was null instead");
-      }
-    } catch (ReplicationTableOfflineException e) {
-      log.trace("Not attempting to remove complete replication records as the"
-          + " table ({}) isn't yet online", ReplicationTable.NAME);
-      return;
-    }
-
-    bs.setRanges(Collections.singleton(new Range()));
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    StatusSection.limit(bs);
-    WorkSection.limit(bs);
-    bs.addScanIterator(cfg);
-
-    long recordsRemoved = 0;
-    long startTime = System.nanoTime();
-    Duration duration;
-    try {
-      recordsRemoved = removeCompleteRecords(client, bs, bw);
-    } finally {
-      bs.close();
-      try {
-        bw.close();
-      } catch (MutationsRejectedException e) {
-        log.error("Error writing mutations to {}, will retry", ReplicationTable.NAME, e);
-      }
-      duration = Duration.ofNanos(System.nanoTime() - startTime);
-    }
-
-    log.info("Removed {} complete replication entries from the table {} in {}", recordsRemoved,
-        ReplicationTable.NAME, duration);
-  }
-
-  /**
-   * Removes {@link Status} records read from the given {@code bs} and writes a delete, using the
-   * given {@code bw}, when that {@link Status} is fully replicated and closed, as defined by
-   * {@link StatusUtil#isSafeForRemoval(org.apache.accumulo.server.replication.proto.Replication.Status)}.
-   *
-   * @param client Accumulo client
-   * @param bs A BatchScanner to read replication status records from
-   * @param bw A BatchWriter to write deletes to
-   * @return Number of records removed
-   */
-  protected long removeCompleteRecords(AccumuloClient client, BatchScanner bs, BatchWriter bw) {
-    Text row = new Text(), colf = new Text(), colq = new Text();
-    long recordsRemoved = 0;
-
-    // For each row in the replication table
-    for (Entry<Key,Value> rowEntry : bs) {
-      SortedMap<Key,Value> columns;
-      try {
-        columns = WholeRowIterator.decodeRow(rowEntry.getKey(), rowEntry.getValue());
-      } catch (IOException e) {
-        log.error("Could not deserialize {} with WholeRowIterator", rowEntry.getKey().getRow(), e);
-        continue;
-      }
-
-      rowEntry.getKey().getRow(row);
-
-      // Try to remove the row (all or nothing)
-      recordsRemoved += removeRowIfNecessary(bw, columns, row, colf, colq);
-    }
-
-    return recordsRemoved;
-  }
-
-  protected long removeRowIfNecessary(BatchWriter bw, SortedMap<Key,Value> columns, Text row,
-      Text colf, Text colq) {
-    long recordsRemoved = 0;
-    if (columns.isEmpty()) {
-      return recordsRemoved;
-    }
-
-    Mutation m = new Mutation(row);
-    Map<TableId,Long> tableToTimeCreated = new HashMap<>();
-    for (Entry<Key,Value> entry : columns.entrySet()) {
-      Status status = null;
-      try {
-        status = Status.parseFrom(entry.getValue().get());
-      } catch (InvalidProtocolBufferException e) {
-        log.error("Encountered unparsable protobuf for key: {}",
-            entry.getKey().toStringNoTruncate());
-        continue;
-      }
-
-      // If a column in the row isn't ready for removal, we keep the whole row
-      if (!StatusUtil.isSafeForRemoval(status)) {
-        return 0L;
-      }
-
-      Key k = entry.getKey();
-      k.getColumnFamily(colf);
-      k.getColumnQualifier(colq);
-
-      log.debug("Removing {} {}:{} from replication table", row, colf, colq);
-
-      m.putDelete(colf, colq);
-
-      TableId tableId;
-      if (StatusSection.NAME.equals(colf)) {
-        tableId = TableId.of(colq.toString());
-      } else if (WorkSection.NAME.equals(colf)) {
-        ReplicationTarget target = ReplicationTarget.from(colq);
-        tableId = target.getSourceTableId();
-      } else {
-        throw new RuntimeException("Got unexpected column");
-      }
-
-      if (status.hasCreatedTime()) {
-        Long timeClosed = tableToTimeCreated.get(tableId);
-        if (timeClosed == null) {
-          tableToTimeCreated.put(tableId, status.getCreatedTime());
-        } else if (timeClosed != status.getCreatedTime()) {
-          log.warn("Found multiple values for timeClosed for {}: {} and {}", row, timeClosed,
-              status.getCreatedTime());
-        }
-      }
-
-      recordsRemoved++;
-    }
-
-    List<Mutation> mutations = new ArrayList<>();
-    mutations.add(m);
-    for (Entry<TableId,Long> entry : tableToTimeCreated.entrySet()) {
-      log.info("Removing order mutation for table {} at {} for {}", entry.getKey(),
-          entry.getValue(), row);
-      Mutation orderMutation = OrderSection.createMutation(row.toString(), entry.getValue());
-      orderMutation.putDelete(OrderSection.NAME, new Text(entry.getKey().canonical()));
-      mutations.add(orderMutation);
-    }
-
-    // Send the mutation deleting all the columns at once.
-    // If we send them not as a single Mutation, we run the risk of having some of them be applied
-    // which would mean that we might accidentally re-replicate data. We want to get rid of them all
-    // at once
-    // or not at all.
-    try {
-      bw.addMutations(mutations);
-      bw.flush();
-    } catch (MutationsRejectedException e) {
-      log.error("Could not submit mutation to remove columns for {} in replication table", row, e);
-      return 0L;
-    }
-
-    return recordsRemoved;
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java
deleted file mode 100644
index c1ee41b..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java
+++ /dev/null
@@ -1,130 +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.manager.replication;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.manager.Manager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
-
-/**
- * Daemon wrapper around the {@link WorkMaker} that separates it from the Manager
- */
-@Deprecated
-public class ReplicationDriver implements Runnable {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationDriver.class);
-
-  private final Manager manager;
-  private final AccumuloConfiguration conf;
-
-  private WorkMaker workMaker;
-  private StatusMaker statusMaker;
-  private FinishedWorkUpdater finishedWorkUpdater;
-  private RemoveCompleteReplicationRecords rcrr;
-  private AccumuloClient client;
-
-  public ReplicationDriver(Manager manager) {
-    this.manager = manager;
-    this.conf = manager.getConfiguration();
-  }
-
-  @Override
-  public void run() {
-
-    long millisToWait = conf.getTimeInMillis(Property.REPLICATION_DRIVER_DELAY);
-    log.debug("Waiting {}ms before starting main replication loop", millisToWait);
-    UtilWaitThread.sleep(millisToWait);
-
-    log.debug("Starting replication loop");
-
-    while (manager.stillManager()) {
-      if (workMaker == null) {
-        client = manager.getContext();
-        statusMaker = new StatusMaker(client, manager.getVolumeManager());
-        workMaker = new WorkMaker(manager.getContext(), client);
-        finishedWorkUpdater = new FinishedWorkUpdater(client);
-        rcrr = new RemoveCompleteReplicationRecords(client);
-      }
-
-      Span span = TraceUtil.startSpan(this.getClass(), "managerReplicationDriver");
-      try (Scope scope = span.makeCurrent()) {
-
-        // Make status markers from replication records in metadata, removing entries in
-        // metadata which are no longer needed (closed records)
-        // This will end up creating the replication table too
-        try {
-          statusMaker.run();
-        } catch (Exception e) {
-          TraceUtil.setException(span, e, false);
-          log.error("Caught Exception trying to create Replication status records", e);
-        }
-
-        // Tell the work maker to make work
-        try {
-          workMaker.run();
-        } catch (Exception e) {
-          TraceUtil.setException(span, e, false);
-          log.error("Caught Exception trying to create Replication work records", e);
-        }
-
-        // Update the status records from the work records
-        try {
-          finishedWorkUpdater.run();
-        } catch (Exception e) {
-          TraceUtil.setException(span, e, false);
-          log.error(
-              "Caught Exception trying to update Replication records using finished work records",
-              e);
-        }
-
-        // Clean up records we no longer need.
-        // It must be running at the same time as the StatusMaker or WorkMaker
-        // So it's important that we run these sequentially and not concurrently
-        try {
-          rcrr.run();
-        } catch (Exception e) {
-          TraceUtil.setException(span, e, false);
-          log.error("Caught Exception trying to remove finished Replication records", e);
-        }
-
-      } catch (Exception e) {
-        TraceUtil.setException(span, e, true);
-        throw e;
-      } finally {
-        span.end();
-      }
-
-      // Sleep for a bit
-      long sleepMillis = conf.getTimeInMillis(Property.MANAGER_REPLICATION_SCAN_INTERVAL);
-      log.trace("Sleeping for {}ms before re-running", sleepMillis);
-      try {
-        Thread.sleep(sleepMillis);
-      } catch (InterruptedException e) {
-        log.error("Interrupted while sleeping", e);
-      }
-    }
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/SequentialWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/SequentialWorkAssigner.java
deleted file mode 100644
index 8ddf4d5..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/SequentialWorkAssigner.java
+++ /dev/null
@@ -1,229 +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.manager.replication;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.hadoop.fs.Path;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Creates work in ZK which is <code>filename.serialized_ReplicationTarget =&gt; filename</code>,
- * but replicates files in the order in which they were created.
- * <p>
- * The intent is to ensure that WALs are replayed in the same order on the peer in which they were
- * applied on the primary.
- */
-@Deprecated
-public class SequentialWorkAssigner extends DistributedWorkQueueWorkAssigner {
-  private static final Logger log = LoggerFactory.getLogger(SequentialWorkAssigner.class);
-  private static final String NAME = "Sequential Work Assigner";
-
-  // @formatter:off
-  /*
-   * {
-   *    peer1 => {sourceTableId1 => work_queue_key1, sourceTableId2 => work_queue_key2, ...}
-   *    peer2 => {sourceTableId1 => work_queue_key1, sourceTableId3 => work_queue_key4, ...}
-   *    ...
-   * }
-   */
-  // @formatter:on
-  private Map<String,Map<TableId,String>> queuedWorkByPeerName;
-
-  public SequentialWorkAssigner() {}
-
-  public SequentialWorkAssigner(AccumuloConfiguration conf, AccumuloClient client) {
-    configure(conf, client);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  protected void setQueuedWork(Map<String,Map<TableId,String>> queuedWork) {
-    this.queuedWorkByPeerName = queuedWork;
-  }
-
-  /**
-   * Initialize the queuedWork set with the work already sent out
-   */
-  @Override
-  protected void initializeQueuedWork() {
-    if (queuedWorkByPeerName != null) {
-      return;
-    }
-
-    queuedWorkByPeerName = new HashMap<>();
-    List<String> existingWork;
-    try {
-      existingWork = workQueue.getWorkQueued();
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException("Error reading existing queued replication work", e);
-    }
-
-    log.info("Restoring replication work queue state from zookeeper");
-
-    for (String work : existingWork) {
-      Entry<String,ReplicationTarget> entry =
-          DistributedWorkQueueWorkAssignerHelper.fromQueueKey(work);
-      String filename = entry.getKey();
-      String peerName = entry.getValue().getPeerName();
-      TableId sourceTableId = entry.getValue().getSourceTableId();
-
-      log.debug("In progress replication of {} from table with ID {} to peer {}", filename,
-          sourceTableId, peerName);
-
-      queuedWorkByPeerName.computeIfAbsent(peerName, k -> new HashMap<>()).put(sourceTableId, work);
-    }
-  }
-
-  /**
-   * Iterate over the queued work to remove entries that have been completed.
-   */
-  @Override
-  protected void cleanupFinishedWork() {
-    final Iterator<Entry<String,Map<TableId,String>>> queuedWork =
-        queuedWorkByPeerName.entrySet().iterator();
-    final InstanceId instanceId = client.instanceOperations().getInstanceId();
-
-    int elementsRemoved = 0;
-    // Check the status of all the work we've queued up
-    while (queuedWork.hasNext()) {
-      // {peer -> {tableId -> workKey, tableId -> workKey, ... }, peer -> ...}
-      Entry<String,Map<TableId,String>> workForPeer = queuedWork.next();
-
-      // TableID to workKey (filename and ReplicationTarget)
-      Map<TableId,String> queuedReplication = workForPeer.getValue();
-
-      Iterator<Entry<TableId,String>> iter = queuedReplication.entrySet().iterator();
-      // Loop over every target we need to replicate this file to, removing the target when
-      // the replication task has finished
-      while (iter.hasNext()) {
-        // tableID -> workKey
-        Entry<TableId,String> entry = iter.next();
-        // Null equates to the work for this target was finished
-        if (zooCache.get(ZooUtil.getRoot(instanceId) + ReplicationConstants.ZOO_WORK_QUEUE + "/"
-            + entry.getValue()) == null) {
-          log.debug("Removing {} from work assignment state", entry.getValue());
-          iter.remove();
-          elementsRemoved++;
-        }
-      }
-    }
-
-    log.info("Removed {} elements from internal workqueue state because the work was complete",
-        elementsRemoved);
-  }
-
-  @Override
-  protected int getQueueSize() {
-    return queuedWorkByPeerName.size();
-  }
-
-  @Override
-  protected boolean shouldQueueWork(ReplicationTarget target) {
-    Map<TableId,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
-    if (queuedWorkForPeer == null) {
-      return true;
-    }
-
-    String queuedWork = queuedWorkForPeer.get(target.getSourceTableId());
-
-    // If we have no work for the local table to the given peer, submit some!
-    return queuedWork == null;
-  }
-
-  @Override
-  protected boolean queueWork(Path path, ReplicationTarget target) {
-    String queueKey = DistributedWorkQueueWorkAssignerHelper.getQueueKey(path.getName(), target);
-    Map<TableId,String> workForPeer =
-        this.queuedWorkByPeerName.computeIfAbsent(target.getPeerName(), k -> new HashMap<>());
-
-    String queuedWork = workForPeer.get(target.getSourceTableId());
-    if (queuedWork == null) {
-      try {
-        workQueue.addWork(queueKey, path.toString());
-        workForPeer.put(target.getSourceTableId(), queueKey);
-      } catch (KeeperException | InterruptedException e) {
-        log.warn("Could not queue work for {} to {}", path, target, e);
-        return false;
-      }
-
-      return true;
-    } else if (queuedWork.startsWith(path.getName())) {
-      log.debug("Not re-queueing work for {} as it has already been queued for replication to {}",
-          path, target);
-      return false;
-    } else {
-      log.debug("Not queueing {} for work as {} must be replicated to {} first", path, queuedWork,
-          target.getPeerName());
-      return false;
-    }
-  }
-
-  @Override
-  protected Set<String> getQueuedWork(ReplicationTarget target) {
-    Map<TableId,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
-    if (queuedWorkForPeer == null) {
-      return Collections.emptySet();
-    }
-
-    String queuedWork = queuedWorkForPeer.get(target.getSourceTableId());
-    if (queuedWork == null) {
-      return Collections.emptySet();
-    } else {
-      return Collections.singleton(queuedWork);
-    }
-  }
-
-  @Override
-  protected void removeQueuedWork(ReplicationTarget target, String queueKey) {
-    Map<TableId,String> queuedWorkForPeer = this.queuedWorkByPeerName.get(target.getPeerName());
-    if (queuedWorkForPeer == null) {
-      log.warn("removeQueuedWork called when no work was queued for {}", target.getPeerName());
-      return;
-    }
-
-    String queuedWork = queuedWorkForPeer.get(target.getSourceTableId());
-    if (queuedWork.equals(queueKey)) {
-      queuedWorkForPeer.remove(target.getSourceTableId());
-    } else {
-      log.warn("removeQueuedWork called on {} with differing queueKeys, expected {} but was {}",
-          target, queueKey, queuedWork);
-      return;
-    }
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java
deleted file mode 100644
index d5bcc9e..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java
+++ /dev/null
@@ -1,304 +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.manager.replication;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-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.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-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.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
-
-/**
- * Reads replication records from the metadata table and creates status records in the replication
- * table. Deletes the record from the metadata table when it's closed.
- */
-@Deprecated
-public class StatusMaker {
-  private static final Logger log = LoggerFactory.getLogger(StatusMaker.class);
-
-  private final AccumuloClient client;
-  private final VolumeManager fs;
-
-  private BatchWriter replicationWriter, metadataWriter;
-  private String sourceTableName = MetadataTable.NAME;
-
-  public StatusMaker(AccumuloClient client, VolumeManager fs) {
-    this.client = client;
-    this.fs = fs;
-  }
-
-  /**
-   * Not for public use -- visible only for testing
-   * <p>
-   * Used to read records from a table other than 'metadata'
-   *
-   * @param table The table to read from
-   */
-  public void setSourceTableName(String table) {
-    this.sourceTableName = table;
-  }
-
-  public void run() {
-    Span span = TraceUtil.startSpan(this.getClass(), "replicationStatusMaker");
-    try (Scope scope = span.makeCurrent()) {
-      // Read from a source table (typically accumulo.metadata)
-      final Scanner s;
-      try {
-        s = client.createScanner(sourceTableName, Authorizations.EMPTY);
-      } catch (TableNotFoundException e) {
-        throw new RuntimeException(e);
-      }
-
-      // Only pull replication records
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      s.setRange(ReplicationSection.getRange());
-
-      Text file = new Text();
-      for (Entry<Key,Value> entry : s) {
-        // Get a writer to the replication table
-        if (replicationWriter == null) {
-          // Ensures table is online
-          try {
-            ReplicationTable.setOnline(client);
-            replicationWriter = ReplicationTable.getBatchWriter(client);
-          } catch (ReplicationTableOfflineException | AccumuloSecurityException
-              | AccumuloException e) {
-            log.warn("Replication table did not come online");
-            replicationWriter = null;
-            return;
-          }
-        }
-        // Extract the useful bits from the status key
-        ReplicationSection.getFile(entry.getKey(), file);
-        TableId tableId = ReplicationSection.getTableId(entry.getKey());
-
-        Status status;
-        try {
-          status = Status.parseFrom(entry.getValue().get());
-        } catch (InvalidProtocolBufferException e) {
-          log.warn("Could not deserialize protobuf for {}", file);
-          continue;
-        }
-
-        log.debug("Creating replication status record for {} on table {} with {}.", file, tableId,
-            ProtobufUtil.toString(status));
-
-        Span childSpan = TraceUtil.startSpan(this.getClass(), "createStatusMutations");
-        try (Scope childScope = span.makeCurrent()) {
-          // Create entries in the replication table from the metadata table
-          if (!addStatusRecord(file, tableId, entry.getValue())) {
-            continue;
-          }
-        } catch (Exception e) {
-          TraceUtil.setException(childSpan, e, true);
-          throw e;
-        } finally {
-          childSpan.end();
-        }
-
-        if (status.getClosed()) {
-          Span closedSpan = TraceUtil.startSpan(this.getClass(), "recordStatusOrder");
-          try (Scope childScope = closedSpan.makeCurrent()) {
-            if (!addOrderRecord(file, tableId, status, entry.getValue())) {
-              continue;
-            }
-          } catch (Exception e) {
-            TraceUtil.setException(closedSpan, e, true);
-            throw e;
-          } finally {
-            closedSpan.end();
-          }
-
-          Span deleteSpan = TraceUtil.startSpan(this.getClass(), "deleteClosedStatus");
-          try (Scope childScope = deleteSpan.makeCurrent()) {
-            deleteStatusRecord(entry.getKey());
-          } catch (Exception e) {
-            TraceUtil.setException(deleteSpan, e, true);
-            throw e;
-          } finally {
-            deleteSpan.end();
-          }
-        }
-      }
-    } catch (Exception e) {
-      TraceUtil.setException(span, e, true);
-      throw e;
-    } finally {
-      span.end();
-    }
-  }
-
-  /**
-   * Create a status record in the replication table
-   */
-  protected boolean addStatusRecord(Text file, TableId tableId, Value v) {
-    try {
-      Mutation m = new Mutation(file);
-      m.put(StatusSection.NAME, new Text(tableId.canonical()), v);
-
-      try {
-        replicationWriter.addMutation(m);
-      } catch (MutationsRejectedException e) {
-        log.warn("Failed to write work mutations for replication, will retry", e);
-        return false;
-      }
-    } finally {
-      try {
-        replicationWriter.flush();
-      } catch (MutationsRejectedException e) {
-        log.warn("Failed to write work mutations for replication, will retry", e);
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  /**
-   * Create a record to track when the file was closed to ensure that replication preference is
-   * given to files that have been closed the longest and allow the work assigner to try to
-   * replicate in order that data was ingested (avoid replay in different order)
-   *
-   * @param file File being replicated
-   * @param tableId Table ID the file was used by
-   * @param stat Status msg
-   * @param value Serialized version of the Status msg
-   */
-  protected boolean addOrderRecord(Text file, TableId tableId, Status stat, Value value) {
-    try {
-      if (!stat.hasCreatedTime()) {
-        try {
-          // If the createdTime is not set, work around the issue by retrieving the WAL creation
-          // time
-          // from HDFS (or the current time if the WAL does not exist). See ACCUMULO-4751
-          long createdTime = setAndGetCreatedTime(new Path(file.toString()), tableId.toString());
-          stat = Status.newBuilder(stat).setCreatedTime(createdTime).build();
-          value = ProtobufUtil.toValue(stat);
-          log.debug("Status was lacking createdTime, set to {} for {}", createdTime, file);
-        } catch (IOException e) {
-          log.warn("Failed to get file status, will retry", e);
-          return false;
-        } catch (MutationsRejectedException e) {
-          log.warn("Failed to write status mutation for replication, will retry", e);
-          return false;
-        }
-      }
-
-      log.info("Creating order record for {} for {} with {}", file, tableId,
-          ProtobufUtil.toString(stat));
-
-      Mutation m = OrderSection.createMutation(file.toString(), stat.getCreatedTime());
-      OrderSection.add(m, tableId, value);
-
-      try {
-        replicationWriter.addMutation(m);
-      } catch (MutationsRejectedException e) {
-        log.warn("Failed to write order mutation for replication, will retry", e);
-        return false;
-      }
-    } finally {
-      try {
-        replicationWriter.flush();
-      } catch (MutationsRejectedException e) {
-        log.warn("Failed to write order mutation for replication, will retry", e);
-        return false;
-      }
-    }
-
-    return true;
-  }
-
-  /**
-   * Because there is only one active Manager, and thus one active StatusMaker, the only safe time
-   * that we can issue the delete for a Status which is closed is immediately after writing it to
-   * the replication table.
-   * <p>
-   * If we try to defer and delete these entries in another thread/process, we will have no
-   * assurance that the Status message was propagated to the replication table. It is easiest, in
-   * terms of concurrency, to do this all in one step.
-   *
-   * @param k The Key to delete
-   */
-  protected void deleteStatusRecord(Key k) {
-    log.debug("Deleting {} from metadata table as it's no longer needed", k.toStringNoTruncate());
-    if (metadataWriter == null) {
-      try {
-        metadataWriter = client.createBatchWriter(sourceTableName);
-      } catch (TableNotFoundException e) {
-        throw new RuntimeException("Metadata table doesn't exist");
-      }
-    }
-
-    try {
-      Mutation m = new Mutation(k.getRow());
-      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
-      metadataWriter.addMutation(m);
-      metadataWriter.flush();
-    } catch (MutationsRejectedException e) {
-      log.warn("Failed to delete status mutations for metadata table, will retry", e);
-    }
-  }
-
-  private long setAndGetCreatedTime(Path file, String tableId)
-      throws IOException, MutationsRejectedException {
-    long createdTime;
-    if (fs.exists(file)) {
-      createdTime = fs.getFileStatus(file).getModificationTime();
-    } else {
-      createdTime = System.currentTimeMillis();
-    }
-
-    Status status = Status.newBuilder().setCreatedTime(createdTime).build();
-    Mutation m = new Mutation(new Text(ReplicationSection.getRowPrefix() + file));
-    m.put(ReplicationSection.COLF, new Text(tableId), ProtobufUtil.toValue(status));
-    replicationWriter.addMutation(m);
-    replicationWriter.flush();
-
-    return createdTime;
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java
deleted file mode 100644
index 0c905db..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java
+++ /dev/null
@@ -1,185 +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.manager.replication;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.hadoop.fs.Path;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Read work records from the replication table, create work entries for other nodes to complete.
- * <p>
- * Uses the DistributedWorkQueue to make the work available for any tserver. This approach does not
- * consider the locality of the tabletserver performing the work in relation to the data being
- * replicated (local HDFS blocks).
- * <p>
- * The implementation allows for multiple tservers to concurrently replicate data to peer(s),
- * however it is possible that data for a table is replayed on the peer in a different order than
- * the manager. The {@link SequentialWorkAssigner} should be used if this must be guaranteed at the
- * cost of replication throughput.
- */
-@Deprecated
-public class UnorderedWorkAssigner extends DistributedWorkQueueWorkAssigner {
-  private static final Logger log = LoggerFactory.getLogger(UnorderedWorkAssigner.class);
-  private static final String NAME = "Unordered Work Assigner";
-
-  private Set<String> queuedWork;
-
-  public UnorderedWorkAssigner() {}
-
-  public UnorderedWorkAssigner(AccumuloConfiguration conf, AccumuloClient client) {
-    configure(conf, client);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  protected Set<String> getQueuedWork() {
-    return queuedWork;
-  }
-
-  protected void setQueuedWork(Set<String> queuedWork) {
-    this.queuedWork = queuedWork;
-  }
-
-  /**
-   * Initialize the queuedWork set with the work already sent out
-   */
-  @Override
-  protected void initializeQueuedWork() {
-    if (queuedWork != null) {
-      return;
-    }
-
-    queuedWork = new HashSet<>();
-    while (true) {
-      try {
-        queuedWork.addAll(workQueue.getWorkQueued());
-        return;
-      } catch (KeeperException e) {
-        if (KeeperException.Code.NONODE.equals(e.code())) {
-          log.warn("Could not find ZK root for replication work queue, will retry", e);
-          sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
-          continue;
-        }
-
-        log.error("Error reading existing queued replication work from ZooKeeper", e);
-        throw new RuntimeException("Error reading existing queued replication work from ZooKeeper",
-            e);
-      } catch (InterruptedException e) {
-        log.error("Error reading existing queued replication work from ZooKeeper", e);
-        throw new RuntimeException("Error reading existing queued replication work from ZooKeeper",
-            e);
-      }
-    }
-  }
-
-  /**
-   * Distribute the work for the given path with filename
-   *
-   * @param path Path to the file being replicated
-   * @param target Target for the file to be replicated to
-   */
-  @Override
-  protected boolean queueWork(Path path, ReplicationTarget target) {
-    String queueKey = DistributedWorkQueueWorkAssignerHelper.getQueueKey(path.getName(), target);
-    if (queuedWork.contains(queueKey)) {
-      log.debug("{} is already queued to be replicated to {}, not re-queueing", path, target);
-      return false;
-    }
-
-    try {
-      log.debug("Queued work for {} and {}", queueKey, path);
-      workQueue.addWork(queueKey, path.toString());
-      queuedWork.add(queueKey);
-    } catch (KeeperException | InterruptedException e) {
-      log.warn("Could not queue work for {}", path, e);
-      return false;
-    }
-
-    return true;
-  }
-
-  /**
-   * Iterate over the queued work to remove entries that have been completed.
-   */
-  @Override
-  protected void cleanupFinishedWork() {
-    final Iterator<String> work = queuedWork.iterator();
-    final InstanceId instanceId = client.instanceOperations().getInstanceId();
-    while (work.hasNext()) {
-      String filename = work.next();
-      // Null equates to the work was finished
-      if (zooCache
-          .get(ZooUtil.getRoot(instanceId) + ReplicationConstants.ZOO_WORK_QUEUE + "/" + filename)
-          == null) {
-        work.remove();
-      }
-    }
-  }
-
-  @Override
-  protected boolean shouldQueueWork(ReplicationTarget target) {
-    // We don't care about ordering, just replicate it all
-    return true;
-  }
-
-  @Override
-  protected int getQueueSize() {
-    return this.queuedWork.size();
-  }
-
-  @Override
-  protected Set<String> getQueuedWork(ReplicationTarget target) {
-    String desiredQueueKeySuffix =
-        DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getPeerName()
-            + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
-            + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
-    Set<String> queuedWorkForTarget = new HashSet<>();
-    for (String queuedWork : this.queuedWork) {
-      if (queuedWork.endsWith(desiredQueueKeySuffix)) {
-        queuedWorkForTarget.add(queuedWork);
-      }
-    }
-
-    return queuedWorkForTarget;
-  }
-
-  @Override
-  protected void removeQueuedWork(ReplicationTarget target, String queueKey) {
-    this.queuedWork.remove(queueKey);
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.java
deleted file mode 100644
index bde35db..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.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.manager.replication;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.server.replication.WorkAssigner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Driver for a {@link WorkAssigner}
- */
-@Deprecated
-public class WorkDriver implements Runnable {
-  private static final Logger log = LoggerFactory.getLogger(WorkDriver.class);
-
-  private Manager manager;
-  private AccumuloClient client;
-  private AccumuloConfiguration conf;
-
-  private WorkAssigner assigner;
-  private String assignerImplName;
-
-  public WorkDriver(Manager manager) {
-    this.manager = manager;
-    this.client = manager.getContext();
-    this.conf = manager.getConfiguration();
-    configureWorkAssigner();
-  }
-
-  protected void configureWorkAssigner() {
-    String workAssignerClass = conf.get(Property.REPLICATION_WORK_ASSIGNER);
-
-    if (assigner == null || !assigner.getClass().getName().equals(workAssignerClass)) {
-      log.info("Initializing work assigner implementation of {}", workAssignerClass);
-
-      try {
-        Class<?> clz = Class.forName(workAssignerClass);
-        Class<? extends WorkAssigner> workAssignerClz = clz.asSubclass(WorkAssigner.class);
-        this.assigner = workAssignerClz.getDeclaredConstructor().newInstance();
-      } catch (ReflectiveOperationException e) {
-        log.error("Could not instantiate configured work assigner {}", workAssignerClass, e);
-        throw new RuntimeException(e);
-      }
-
-      this.assigner.configure(conf, client);
-      this.assignerImplName = assigner.getClass().getName();
-    }
-  }
-
-  public String getName() {
-    return assigner.getName();
-  }
-
-  @Override
-  public void run() {
-    log.info("Starting replication work assignment thread using {}", assignerImplName);
-
-    while (manager.stillManager()) {
-      // Assign the work using the configured implementation
-      try {
-        assigner.assignWork();
-      } catch (Exception e) {
-        log.error("Error while assigning work", e);
-      }
-
-      long sleepTime = conf.getTimeInMillis(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP);
-      log.trace("Sleeping {} ms before next work assignment", sleepTime);
-      sleepUninterruptibly(sleepTime, TimeUnit.MILLISECONDS);
-
-      // After each loop, make sure that the WorkAssigner implementation didn't change
-      configureWorkAssigner();
-    }
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkMaker.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkMaker.java
deleted file mode 100644
index dd5e780..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkMaker.java
+++ /dev/null
@@ -1,225 +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.manager.replication;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-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.accumulo.core.replication.ReplicationSchema;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
-
-/**
- * Reads replication records from the replication table and creates work records which include
- * target replication system information.
- */
-@Deprecated
-public class WorkMaker {
-  private static final Logger log = LoggerFactory.getLogger(WorkMaker.class);
-
-  private final ServerContext context;
-  private AccumuloClient client;
-
-  private BatchWriter writer;
-
-  public WorkMaker(ServerContext context, AccumuloClient client) {
-    this.context = context;
-    this.client = client;
-  }
-
-  public void run() {
-    if (!ReplicationTable.isOnline(client)) {
-      log.trace("Replication table is not yet online");
-      return;
-    }
-
-    Span span = TraceUtil.startSpan(this.getClass(), "replicationWorkMaker");
-    try (Scope scope = span.makeCurrent()) {
-      final Scanner s;
-      try {
-        s = ReplicationTable.getScanner(client);
-        if (writer == null) {
-          setBatchWriter(ReplicationTable.getBatchWriter(client));
-        }
-      } catch (ReplicationTableOfflineException e) {
-        TraceUtil.setException(span, e, false);
-        log.warn("Replication table was online, but not anymore");
-        writer = null;
-        return;
-      }
-
-      // Only pull records about data that has been ingested and is ready for replication
-      StatusSection.limit(s);
-
-      TableConfiguration tableConf;
-
-      Text file = new Text();
-      for (Entry<Key,Value> entry : s) {
-        // Extract the useful bits from the status key
-        ReplicationSchema.StatusSection.getFile(entry.getKey(), file);
-        TableId tableId = ReplicationSchema.StatusSection.getTableId(entry.getKey());
-        log.debug("Processing replication status record for {} on table {}", file, tableId);
-
-        Status status;
-        try {
-          status = Status.parseFrom(entry.getValue().get());
-        } catch (InvalidProtocolBufferException e) {
-          log.error("Could not parse protobuf for {} from table {}", file, tableId);
-          continue;
-        }
-
-        // Don't create the record if we have nothing to do.
-        // TODO put this into a filter on serverside
-        if (!shouldCreateWork(status)) {
-          log.debug("Not creating work: {}", status);
-          continue;
-        }
-
-        // Get the table configuration for the table specified by the status record
-        tableConf = context.getTableConfiguration(tableId);
-
-        // getTableConfiguration(String) returns null if the table no longer exists
-        if (tableConf == null) {
-          continue;
-        }
-
-        // Pull the relevant replication targets
-        // TODO Cache this instead of pulling it every time
-        Map<String,String> replicationTargets = getReplicationTargets(tableConf);
-
-        // If we have targets, we need to make a work record
-        // TODO Don't replicate if it's a only a newFile entry (nothing to replicate yet)
-        // -- Another scanner over the WorkSection can make this relatively cheap
-        if (replicationTargets.isEmpty()) {
-          log.warn("No configured targets for table with ID {}", tableId);
-        } else {
-          Span childSpan = TraceUtil.startSpan(this.getClass(), "createWorkMutations");
-          try (Scope childScope = childSpan.makeCurrent()) {
-            addWorkRecord(file, entry.getValue(), replicationTargets, tableId);
-          } catch (Exception e) {
-            TraceUtil.setException(childSpan, e, true);
-            throw e;
-          } finally {
-            childSpan.end();
-          }
-        }
-      }
-    } catch (Exception e) {
-      TraceUtil.setException(span, e, true);
-      throw e;
-    } finally {
-      span.end();
-    }
-  }
-
-  protected void setBatchWriter(BatchWriter bw) {
-    this.writer = bw;
-  }
-
-  protected Map<String,String> getReplicationTargets(TableConfiguration tableConf) {
-    final Map<String,String> props =
-        tableConf.getAllPropertiesWithPrefix(Property.TABLE_REPLICATION_TARGET);
-    final Map<String,String> targets = new HashMap<>();
-    final int propKeyLength = Property.TABLE_REPLICATION_TARGET.getKey().length();
-
-    for (Entry<String,String> prop : props.entrySet()) {
-      targets.put(prop.getKey().substring(propKeyLength), prop.getValue());
-    }
-
-    return targets;
-  }
-
-  /**
-   * @return Should a Work entry be created for this status
-   */
-  protected boolean shouldCreateWork(Status status) {
-    // Only creating work when there is work to do (regardless of closed status) is safe
-    // as long as the ReplicaSystem implementation is correctly observing
-    // that a file is completely replicated only when the file is closed
-    return StatusUtil.isWorkRequired(status);
-  }
-
-  protected void addWorkRecord(Text file, Value v, Map<String,String> targets,
-      TableId sourceTableId) {
-    log.info("Adding work records for {} to targets {}", file, targets);
-    try {
-      Mutation m = new Mutation(file);
-
-      ReplicationTarget target = new ReplicationTarget();
-      DataOutputBuffer buffer = new DataOutputBuffer();
-      Text t = new Text();
-      for (Entry<String,String> entry : targets.entrySet()) {
-        buffer.reset();
-
-        // Set up the writable
-        target.setPeerName(entry.getKey());
-        target.setRemoteIdentifier(entry.getValue());
-        target.setSourceTableId(sourceTableId);
-        target.write(buffer);
-
-        // Throw it in a text for the mutation
-        t.set(buffer.getData(), 0, buffer.getLength());
-
-        // Add it to the work section
-        WorkSection.add(m, t, v);
-      }
-      try {
-        writer.addMutation(m);
-      } catch (MutationsRejectedException e) {
-        log.warn("Failed to write work mutations for replication, will retry", e);
-      }
-    } catch (IOException e) {
-      log.warn("Failed to serialize data to Text, will retry", e);
-    } finally {
-      try {
-        writer.flush();
-      } catch (MutationsRejectedException e) {
-        log.warn("Failed to write work mutations for replication, will retry", e);
-      }
-    }
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java
index 5cecbfe..a60f8f4 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java
@@ -34,8 +34,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-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.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletLocationState.BadLocationStateException;
 import org.apache.accumulo.core.metadata.TabletState;
@@ -63,52 +62,23 @@
   private final MergeInfo info;
   private int hosted = 0;
   private int unassigned = 0;
-  private int chopped = 0;
-  private int needsToBeChopped = 0;
   private int total = 0;
-  private boolean lowerSplit = false;
-  private boolean upperSplit = false;
 
   public MergeStats(MergeInfo info) {
     this.info = info;
-    if (info.getState().equals(MergeState.NONE)) {
-      return;
-    }
-    if (info.getExtent().endRow() == null) {
-      upperSplit = true;
-    }
-    if (info.getExtent().prevEndRow() == null) {
-      lowerSplit = true;
-    }
   }
 
   public MergeInfo getMergeInfo() {
     return info;
   }
 
-  public void update(KeyExtent ke, TabletState state, boolean chopped, boolean hasWALs) {
+  public void update(KeyExtent ke, TabletState state) {
     if (info.getState().equals(MergeState.NONE)) {
       return;
     }
-    if (!upperSplit && info.getExtent().endRow().equals(ke.prevEndRow())) {
-      log.info("Upper split found: {}", ke.prevEndRow());
-      upperSplit = true;
-    }
-    if (!lowerSplit && info.getExtent().prevEndRow().equals(ke.endRow())) {
-      log.info("Lower split found: {}", ke.endRow());
-      lowerSplit = true;
-    }
     if (!info.overlaps(ke)) {
       return;
     }
-    if (info.needsToBeChopped(ke)) {
-      this.needsToBeChopped++;
-      if (chopped) {
-        if (state.equals(TabletState.HOSTED) || !hasWALs) {
-          this.chopped++;
-        }
-      }
-    }
     this.total++;
     if (state.equals(TabletState.HOSTED)) {
       this.hosted++;
@@ -131,60 +101,29 @@
     log.info("Computing next merge state for {} which is presently {} isDelete : {}",
         info.getExtent(), state, info.isDelete());
     if (state == MergeState.STARTED) {
-      state = MergeState.SPLITTING;
-    }
-    if (state == MergeState.SPLITTING) {
       log.info("{} are hosted, total {}", hosted, total);
       if (!info.isDelete() && total == 1) {
         log.info("Merge range is already contained in a single tablet {}", info.getExtent());
         state = MergeState.COMPLETE;
       } else if (hosted == total) {
-        if (info.isDelete()) {
-          if (!lowerSplit) {
-            log.info("Waiting for {} lower split to occur {}", info, info.getExtent());
-          } else if (!upperSplit) {
-            log.info("Waiting for {} upper split to occur {}", info, info.getExtent());
-          } else {
-            state = MergeState.WAITING_FOR_CHOPPED;
-          }
-        } else {
-          state = MergeState.WAITING_FOR_CHOPPED;
-        }
+        state = MergeState.WAITING_FOR_OFFLINE;
       } else {
         log.info("Waiting for {} hosted tablets to be {} {}", hosted, total, info.getExtent());
       }
     }
-    if (state == MergeState.WAITING_FOR_CHOPPED) {
-      log.info("{} tablets are chopped {}", chopped, info.getExtent());
-      if (chopped == needsToBeChopped) {
-        state = MergeState.WAITING_FOR_OFFLINE;
-      } else {
-        log.info("Waiting for {} chopped tablets to be {} {}", chopped, needsToBeChopped,
-            info.getExtent());
-      }
-    }
     if (state == MergeState.WAITING_FOR_OFFLINE) {
-      if (chopped == needsToBeChopped) {
-        log.info("{} tablets are chopped, {} are offline {}", chopped, unassigned,
-            info.getExtent());
-        if (unassigned == total) {
-          if (verifyMergeConsistency(accumuloClient, manager)) {
-            state = MergeState.MERGING;
-          } else {
-            log.info("Merge consistency check failed {}", info.getExtent());
-          }
+      if (unassigned == total) {
+        if (verifyMergeConsistency(accumuloClient, manager)) {
+          state = MergeState.MERGING;
         } else {
-          log.info("Waiting for {} unassigned tablets to be {} {}", unassigned, total,
-              info.getExtent());
+          log.info("Merge consistency check failed {}", info.getExtent());
         }
       } else {
-        log.warn("Unexpected state: chopped tablets should be {} was {} merge {}", needsToBeChopped,
-            chopped, info.getExtent());
-        // Perhaps a split occurred after we chopped, but before we went offline: start over
-        state = MergeState.WAITING_FOR_CHOPPED;
+        log.info("Waiting for {} unassigned tablets to be {} {}", unassigned, total,
+            info.getExtent());
       }
     }
-    if (state == MergeState.MERGING) {
+    if (state == MergeState.MERGING || state == MergeState.MERGED) {
       if (hosted != 0) {
         // Shouldn't happen
         log.error("Unexpected state: hosted tablets should be zero {} merge {}", hosted,
@@ -195,7 +134,7 @@
         // Shouldn't happen
         log.error("Unexpected state: unassigned tablets should be {} was {} merge {}", total,
             unassigned, info.getExtent());
-        state = MergeState.WAITING_FOR_CHOPPED;
+        state = MergeState.WAITING_FOR_OFFLINE;
       }
       log.info("{} tablets are unassigned {}", unassigned, info.getExtent());
     }
@@ -209,8 +148,9 @@
 
     MergeStats verify = new MergeStats(info);
     KeyExtent extent = info.getExtent();
-    Scanner scanner = accumuloClient
-        .createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner scanner = accumuloClient.createScanner(
+        extent.isMeta() ? AccumuloTable.ROOT.tableName() : AccumuloTable.METADATA.tableName(),
+        Authorizations.EMPTY);
     MetaDataTableScanner.configureScanner(scanner, manager);
     Text start = extent.prevEndRow();
     if (start == null) {
@@ -257,25 +197,23 @@
         }
 
       } else if (!tls.extent.isPreviousExtent(prevExtent)) {
-        log.debug("hole in {}", MetadataTable.NAME);
+        log.debug("hole in {}", AccumuloTable.METADATA.tableName());
         return false;
       }
 
       prevExtent = tls.extent;
 
-      verify.update(tls.extent, tls.getState(manager.onlineTabletServers()), tls.chopped,
-          !tls.walogs.isEmpty());
+      verify.update(tls.extent, tls.getState(manager.onlineTabletServers()));
       // stop when we've seen the tablet just beyond our range
       if (tls.extent.prevEndRow() != null && extent.endRow() != null
           && tls.extent.prevEndRow().compareTo(extent.endRow()) > 0) {
         break;
       }
     }
-    log.debug("chopped {} v.chopped {} unassigned {} v.unassigned {} verify.total {}", chopped,
-        verify.chopped, unassigned, verify.unassigned, verify.total);
+    log.debug("unassigned {} v.unassigned {} verify.total {}", unassigned, verify.unassigned,
+        verify.total);
 
-    return chopped == verify.chopped && unassigned == verify.unassigned
-        && unassigned == verify.total;
+    return unassigned == verify.unassigned && unassigned == verify.total;
   }
 
   @VisibleForTesting
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
index 2134a51..0914e67 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
@@ -26,7 +26,6 @@
 import org.apache.accumulo.core.manager.thrift.ManagerGoalState;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.manager.upgrade.RenameMasterDirInZK;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.security.SecurityUtil;
 
@@ -45,12 +44,12 @@
     try {
       var siteConfig = SiteConfiguration.auto();
       SecurityUtil.serverLogin(siteConfig);
-      var context = new ServerContext(siteConfig);
-      RenameMasterDirInZK.renameMasterDirInZK(context);
-      context.waitForZookeeperAndHdfs();
-      context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE, args[0].getBytes(UTF_8),
-          NodeExistsPolicy.OVERWRITE);
+      try (var context = new ServerContext(siteConfig)) {
+        context.waitForZookeeperAndHdfs();
+        context.getZooReaderWriter().putPersistentData(
+            context.getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE, args[0].getBytes(UTF_8),
+            NodeExistsPolicy.OVERWRITE);
+      }
     } finally {
       SingletonManager.setMode(Mode.CLOSED);
     }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/ChangeTableState.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/ChangeTableState.java
index 146ea96..496565c 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/ChangeTableState.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/ChangeTableState.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.manager.Manager;
 import org.slf4j.LoggerFactory;
@@ -52,8 +53,8 @@
   public long isReady(long tid, Manager env) throws Exception {
     // reserve the table so that this op does not run concurrently with create, clone, or delete
     // table
-    return Utils.reserveNamespace(env, namespaceId, tid, false, true, top)
-        + Utils.reserveTable(env, tableId, tid, true, true, top);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, true, top)
+        + Utils.reserveTable(env, tableId, tid, LockType.WRITE, true, top);
   }
 
   @Override
@@ -64,8 +65,8 @@
     }
 
     env.getTableManager().transitionTableState(tableId, ts, expectedCurrStates);
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
-    Utils.unreserveTable(env, tableId, tid, true);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
+    Utils.unreserveTable(env, tableId, tid, LockType.WRITE);
     LoggerFactory.getLogger(ChangeTableState.class).debug("Changed table state {} {}", tableId, ts);
     env.getEventCoordinator().event("Set table state of %s to %s", tableId, ts);
     return null;
@@ -73,7 +74,7 @@
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
-    Utils.unreserveTable(env, tableId, tid, true);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
+    Utils.unreserveTable(env, tableId, tid, LockType.WRITE);
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/TraceRepo.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/TraceRepo.java
index 06bd812..752bc90 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/TraceRepo.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/TraceRepo.java
@@ -18,13 +18,13 @@
  */
 package org.apache.accumulo.manager.tableOps;
 
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
+
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.fate.Repo;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.manager.Manager;
 
-import com.google.gson.Gson;
-
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
@@ -106,6 +106,6 @@
 
     // Inorder for Gson to work with generic types, the following passes repo.getClass() to Gson.
     // See the Gson javadoc for more info.
-    return repo.getClass() + " " + new Gson().toJson(repo, repo.getClass());
+    return repo.getClass() + " " + GSON.get().toJson(repo, repo.getClass());
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java
index def7437..a92b3b5 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java
@@ -39,6 +39,8 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.DistributedLock;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.FateLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooReservation;
@@ -88,9 +90,9 @@
   static final Lock tableNameLock = new ReentrantLock();
   static final Lock idLock = new ReentrantLock();
 
-  public static long reserveTable(Manager env, TableId tableId, long tid, boolean writeLock,
+  public static long reserveTable(Manager env, TableId tableId, long tid, LockType lockType,
       boolean tableMustExist, TableOperation op) throws Exception {
-    if (getLock(env.getContext(), tableId, tid, writeLock).tryLock()) {
+    if (getLock(env.getContext(), tableId, tid, lockType).tryLock()) {
       if (tableMustExist) {
         ZooReaderWriter zk = env.getContext().getZooReaderWriter();
         if (!zk.exists(env.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId)) {
@@ -99,29 +101,27 @@
         }
       }
       log.info("table {} {} locked for {} operation: {}", tableId, FateTxId.formatTid(tid),
-          (writeLock ? "write" : "read"), op);
+          lockType, op);
       return 0;
     } else {
       return 100;
     }
   }
 
-  public static void unreserveTable(Manager env, TableId tableId, long tid, boolean writeLock) {
-    getLock(env.getContext(), tableId, tid, writeLock).unlock();
-    log.info("table {} {} unlocked for {}", tableId, FateTxId.formatTid(tid),
-        (writeLock ? "write" : "read"));
+  public static void unreserveTable(Manager env, TableId tableId, long tid, LockType lockType) {
+    getLock(env.getContext(), tableId, tid, lockType).unlock();
+    log.info("table {} {} unlocked for {}", tableId, FateTxId.formatTid(tid), lockType);
   }
 
   public static void unreserveNamespace(Manager env, NamespaceId namespaceId, long id,
-      boolean writeLock) {
-    getLock(env.getContext(), namespaceId, id, writeLock).unlock();
-    log.info("namespace {} {} unlocked for {}", namespaceId, FateTxId.formatTid(id),
-        (writeLock ? "write" : "read"));
+      LockType lockType) {
+    getLock(env.getContext(), namespaceId, id, lockType).unlock();
+    log.info("namespace {} {} unlocked for {}", namespaceId, FateTxId.formatTid(id), lockType);
   }
 
   public static long reserveNamespace(Manager env, NamespaceId namespaceId, long id,
-      boolean writeLock, boolean mustExist, TableOperation op) throws Exception {
-    if (getLock(env.getContext(), namespaceId, id, writeLock).tryLock()) {
+      LockType lockType, boolean mustExist, TableOperation op) throws Exception {
+    if (getLock(env.getContext(), namespaceId, id, lockType).tryLock()) {
       if (mustExist) {
         ZooReaderWriter zk = env.getContext().getZooReaderWriter();
         if (!zk.exists(
@@ -131,7 +131,7 @@
         }
       }
       log.info("namespace {} {} locked for {} operation: {}", namespaceId, FateTxId.formatTid(id),
-          (writeLock ? "write" : "read"), op);
+          lockType, op);
       return 0;
     } else {
       return 100;
@@ -161,18 +161,30 @@
   }
 
   private static Lock getLock(ServerContext context, AbstractId<?> id, long tid,
-      boolean writeLock) {
+      LockType lockType) {
     byte[] lockData = FastFormat.toZeroPaddedHex(tid);
     var fLockPath =
         FateLock.path(context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS + "/" + id.canonical());
     FateLock qlock = new FateLock(context.getZooReaderWriter(), fLockPath);
-    Lock lock = DistributedReadWriteLock.recoverLock(qlock, lockData);
-    if (lock == null) {
+    DistributedLock lock = DistributedReadWriteLock.recoverLock(qlock, lockData);
+    if (lock != null) {
+      // Validate the recovered lock type
+      if (lock.getType() != lockType) {
+        throw new IllegalStateException("Unexpected lock type " + lock.getType()
+            + " recovered for transaction " + FateTxId.formatTid(tid) + " on object " + id
+            + ". Expected " + lockType + " lock instead.");
+      }
+    } else {
       DistributedReadWriteLock locker = new DistributedReadWriteLock(qlock, lockData);
-      if (writeLock) {
-        lock = locker.writeLock();
-      } else {
-        lock = locker.readLock();
+      switch (lockType) {
+        case WRITE:
+          lock = locker.writeLock();
+          break;
+        case READ:
+          lock = locker.readLock();
+          break;
+        default:
+          throw new IllegalStateException("Unexpected LockType: " + lockType);
       }
     }
     return lock;
@@ -187,7 +199,7 @@
   }
 
   public static Lock getReadLock(Manager env, AbstractId<?> id, long tid) {
-    return Utils.getLock(env.getContext(), id, tid, false);
+    return Utils.getLock(env.getContext(), id, tid, LockType.READ);
   }
 
   public static void checkNamespaceDoesNotExist(ServerContext context, String namespace,
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
deleted file mode 100644
index ee0694f..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
+++ /dev/null
@@ -1,292 +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.manager.tableOps.bulkVer1;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException;
-import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
-import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.FateTxId;
-import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.manager.tableOps.ManagerRepo;
-import org.apache.accumulo.manager.tableOps.Utils;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.tablets.UniqueNameAllocator;
-import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.MapFile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Bulk import makes requests of tablet servers, and those requests can take a long time. Our
- * communications to the tablet server may fail, so we won't know the status of the request. The
- * manager will repeat failed requests so now there are multiple requests to the tablet server. The
- * tablet server will not execute the request multiple times, so long as the marker it wrote in the
- * metadata table stays there. The manager needs to know when all requests have finished so it can
- * remove the markers. Did it start? Did it finish? We can see that *a* request completed by seeing
- * the flag written into the metadata table, but we won't know if some other rogue thread is still
- * waiting to start a thread and repeat the operation.
- *
- * The manager can ask the tablet server if it has any requests still running. Except the tablet
- * server might have some thread about to start a request, but before it has made any bookkeeping
- * about the request. To prevent problems like this, an Arbitrator is used. Before starting any new
- * request, the tablet server checks the Arbitrator to see if the request is still valid.
- */
-public class BulkImport extends ManagerRepo {
-  public static final String FAILURES_TXT = "failures.txt";
-
-  private static final long serialVersionUID = 1L;
-
-  private static final Logger log = LoggerFactory.getLogger(BulkImport.class);
-
-  private TableId tableId;
-  private String sourceDir;
-  private String errorDir;
-  private boolean setTime;
-
-  public BulkImport(TableId tableId, String sourceDir, String errorDir, boolean setTime) {
-    this.tableId = tableId;
-    this.sourceDir = sourceDir;
-    this.errorDir = errorDir;
-    this.setTime = setTime;
-  }
-
-  @Override
-  public long isReady(long tid, Manager manager) throws Exception {
-    if (!Utils.getReadLock(manager, tableId, tid).tryLock()) {
-      return 100;
-    }
-
-    manager.getContext().clearTableListCache();
-    if (manager.getContext().getTableState(tableId) == TableState.ONLINE) {
-      long reserve1, reserve2;
-      reserve1 = reserve2 = Utils.reserveHdfsDirectory(manager, sourceDir, tid);
-      if (reserve1 == 0) {
-        reserve2 = Utils.reserveHdfsDirectory(manager, errorDir, tid);
-      }
-      return reserve2;
-    } else {
-      throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
-          TableOperation.BULK_IMPORT, TableOperationExceptionType.OFFLINE, null);
-    }
-  }
-
-  @Override
-  public Repo<Manager> call(long tid, Manager manager) throws Exception {
-    String fmtTid = FateTxId.formatTid(tid);
-
-    log.debug(" {} sourceDir {}", fmtTid, sourceDir);
-
-    Utils.getReadLock(manager, tableId, tid).lock();
-
-    // check that the error directory exists and is empty
-    VolumeManager fs = manager.getVolumeManager();
-
-    Path errorPath = new Path(errorDir);
-    FileStatus errorStatus = null;
-    try {
-      errorStatus = fs.getFileStatus(errorPath);
-    } catch (FileNotFoundException ex) {
-      // ignored
-    }
-    if (errorStatus == null) {
-      throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
-          TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
-          errorDir + " does not exist");
-    }
-    if (!errorStatus.isDirectory()) {
-      throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
-          TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
-          errorDir + " is not a directory");
-    }
-    if (fs.listStatus(errorPath).length != 0) {
-      throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
-          TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
-          errorDir + " is not empty");
-    }
-
-    ZooArbitrator.start(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
-    manager.updateBulkImportStatus(sourceDir, BulkImportState.MOVING);
-    // move the files into the directory
-    try {
-      String bulkDir = prepareBulkImport(manager.getContext(), fs, sourceDir, tableId, tid);
-      log.debug(" {} bulkDir {}", tid, bulkDir);
-      return new LoadFiles(tableId, sourceDir, bulkDir, errorDir, setTime);
-    } catch (IOException ex) {
-      log.error("error preparing the bulk import directory", ex);
-      throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
-          TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_INPUT_DIRECTORY,
-          sourceDir + ": " + ex);
-    }
-  }
-
-  private static Path createNewBulkDir(ServerContext context, VolumeManager fs, String sourceDir,
-      TableId tableId) throws IOException {
-    Path tableDir = fs.matchingFileSystem(new Path(sourceDir), context.getTablesDirs());
-    if (tableDir == null) {
-      throw new IOException(
-          sourceDir + " is not in the same file system as any volume configured for Accumulo");
-    }
-
-    Path directory = new Path(tableDir, tableId.canonical());
-    fs.mkdirs(directory);
-
-    // only one should be able to create the lock file
-    // the purpose of the lock file is to avoid a race
-    // condition between the call to fs.exists() and
-    // fs.mkdirs()... if only hadoop had a mkdir() function
-    // that failed when the dir existed
-
-    UniqueNameAllocator namer = context.getUniqueNameAllocator();
-
-    while (true) {
-      Path newBulkDir = new Path(directory, Constants.BULK_PREFIX + namer.getNextName());
-      if (fs.exists(newBulkDir)) {
-        // sanity check
-        throw new IOException("Dir exist when it should not " + newBulkDir);
-      }
-      if (fs.mkdirs(newBulkDir)) {
-        return newBulkDir;
-      }
-      log.warn("Failed to create {} for unknown reason", newBulkDir);
-
-      sleepUninterruptibly(3, TimeUnit.SECONDS);
-    }
-  }
-
-  @VisibleForTesting
-  public static String prepareBulkImport(ServerContext manager, final VolumeManager fs, String dir,
-      TableId tableId, long tid) throws Exception {
-    final Path bulkDir = createNewBulkDir(manager, fs, dir, tableId);
-
-    manager.getAmple().addBulkLoadInProgressFlag(
-        "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName(), tid);
-
-    Path dirPath = new Path(dir);
-    FileStatus[] mapFiles = fs.listStatus(dirPath);
-
-    final UniqueNameAllocator namer = manager.getUniqueNameAllocator();
-
-    AccumuloConfiguration serverConfig = manager.getConfiguration();
-    @SuppressWarnings("deprecation")
-    ExecutorService workers =
-        ThreadPools.getServerThreadPools().createExecutorService(serverConfig, serverConfig
-            .resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
-    List<Future<Exception>> results = new ArrayList<>();
-
-    for (FileStatus file : mapFiles) {
-      final FileStatus fileStatus = file;
-      results.add(workers.submit(() -> {
-        try {
-          String[] sa = fileStatus.getPath().getName().split("\\.");
-          String extension = "";
-          if (sa.length > 1) {
-            extension = sa[sa.length - 1];
-
-            if (!FileOperations.getValidExtensions().contains(extension)) {
-              log.warn("{} does not have a valid extension, ignoring", fileStatus.getPath());
-              return null;
-            }
-          } else {
-            // assume it is a map file
-            extension = Constants.MAPFILE_EXTENSION;
-          }
-
-          if (extension.equals(Constants.MAPFILE_EXTENSION)) {
-            if (!fileStatus.isDirectory()) {
-              log.warn("{} is not a map file, ignoring", fileStatus.getPath());
-              return null;
-            }
-
-            if (fileStatus.getPath().getName().equals("_logs")) {
-              log.info("{} is probably a log directory from a map/reduce task, skipping",
-                  fileStatus.getPath());
-              return null;
-            }
-            try {
-              FileStatus dataStatus =
-                  fs.getFileStatus(new Path(fileStatus.getPath(), MapFile.DATA_FILE_NAME));
-              if (dataStatus.isDirectory()) {
-                log.warn("{} is not a map file, ignoring", fileStatus.getPath());
-                return null;
-              }
-            } catch (FileNotFoundException fnfe) {
-              log.warn("{} is not a map file, ignoring", fileStatus.getPath());
-              return null;
-            }
-          }
-
-          String newName = "I" + namer.getNextName() + "." + extension;
-          Path newPath = new Path(bulkDir, newName);
-          try {
-            fs.rename(fileStatus.getPath(), newPath);
-            log.debug("Moved {} to {}", fileStatus.getPath(), newPath);
-          } catch (IOException E1) {
-            log.error("Could not move: {} {}", fileStatus.getPath(), E1.getMessage());
-          }
-
-        } catch (Exception ex) {
-          return ex;
-        }
-        return null;
-      }));
-    }
-    workers.shutdown();
-    while (!workers.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {}
-
-    for (Future<Exception> ex : results) {
-      if (ex.get() != null) {
-        throw ex.get();
-      }
-    }
-    return bulkDir.toString();
-  }
-
-  @Override
-  public void undo(long tid, Manager environment) throws Exception {
-    // unreserve source/error directories
-    Utils.unreserveHdfsDirectory(environment, sourceDir, tid);
-    Utils.unreserveHdfsDirectory(environment, errorDir, tid);
-    Utils.getReadLock(environment, tableId, tid).unlock();
-    ZooArbitrator.cleanup(environment.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java
deleted file mode 100644
index 1ed199a..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java
+++ /dev/null
@@ -1,77 +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.manager.tableOps.bulkVer1;
-
-import java.util.Collections;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.FateTxId;
-import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.gc.ReferenceFile;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.manager.tableOps.ManagerRepo;
-import org.apache.accumulo.manager.tableOps.Utils;
-import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class CleanUpBulkImport extends ManagerRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private static final Logger log = LoggerFactory.getLogger(CleanUpBulkImport.class);
-
-  private TableId tableId;
-  private String source;
-  private String bulk;
-  private String error;
-
-  CleanUpBulkImport(TableId tableId, String source, String bulk, String error) {
-    this.tableId = tableId;
-    this.source = source;
-    this.bulk = bulk;
-    this.error = error;
-  }
-
-  @Override
-  public Repo<Manager> call(long tid, Manager manager) throws Exception {
-    manager.updateBulkImportStatus(source, BulkImportState.CLEANUP);
-    log.debug("removing the bulkDir processing flag file in " + bulk);
-    Path bulkDir = new Path(bulk);
-    Ample ample = manager.getContext().getAmple();
-    ample.removeBulkLoadInProgressFlag(
-        "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
-    ample.putGcFileAndDirCandidates(tableId,
-        Collections.singleton(ReferenceFile.forFile(tableId, bulkDir.toString())));
-    log.debug("removing the metadata table markers for loaded files");
-    ample.removeBulkLoadEntries(tableId, tid, null, null);
-    log.debug("releasing HDFS reservations for " + source + " and " + error);
-    Utils.unreserveHdfsDirectory(manager, source, tid);
-    Utils.unreserveHdfsDirectory(manager, error, tid);
-    Utils.getReadLock(manager, tableId, tid).unlock();
-    log.debug("completing bulkDir import transaction " + FateTxId.formatTid(tid));
-    ZooArbitrator.cleanup(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
-    manager.removeBulkImportStatus(source);
-    return null;
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.java
deleted file mode 100644
index f51f9ce..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.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.manager.tableOps.bulkVer1;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.manager.tableOps.ManagerRepo;
-import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
-
-public class CompleteBulkImport extends ManagerRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private TableId tableId;
-  private String source;
-  private String bulk;
-  private String error;
-
-  public CompleteBulkImport(TableId tableId, String source, String bulk, String error) {
-    this.tableId = tableId;
-    this.source = source;
-    this.bulk = bulk;
-    this.error = error;
-  }
-
-  @Override
-  public Repo<Manager> call(long tid, Manager manager) throws Exception {
-    ZooArbitrator.stop(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
-    return new CopyFailed(tableId, source, bulk, error);
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java
deleted file mode 100644
index 53d5dd4..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java
+++ /dev/null
@@ -1,174 +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.manager.tableOps.bulkVer1;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.Scanner;
-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.dataImpl.KeyExtent;
-import org.apache.accumulo.core.fate.FateTxId;
-import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.metadata.ValidationUtil;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.manager.tableOps.ManagerRepo;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.hadoop.fs.Path;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-class CopyFailed extends ManagerRepo {
-  private static final Logger log = LoggerFactory.getLogger(CopyFailed.class);
-
-  private static final long serialVersionUID = 1L;
-
-  private TableId tableId;
-  private String source;
-  private String bulk;
-  private String error;
-
-  public CopyFailed(TableId tableId, String source, String bulk, String error) {
-    this.tableId = tableId;
-    this.source = source;
-    this.bulk = bulk;
-    this.error = error;
-  }
-
-  @Override
-  public long isReady(long tid, Manager manager) {
-    Set<TServerInstance> finished = new HashSet<>();
-    Set<TServerInstance> running = manager.onlineTabletServers();
-    for (TServerInstance server : running) {
-      try {
-        TServerConnection client = manager.getConnection(server);
-        if (client != null && !client.isActive(tid)) {
-          finished.add(server);
-        }
-      } catch (TException ex) {
-        log.info("Ignoring error trying to check on tid " + FateTxId.formatTid(tid)
-            + " from server " + server + ": " + ex);
-      }
-    }
-    if (finished.containsAll(running)) {
-      return 0;
-    }
-    return 500;
-  }
-
-  @Override
-  public Repo<Manager> call(long tid, Manager manager) throws Exception {
-    // This needs to execute after the arbiter is stopped
-    manager.updateBulkImportStatus(source, BulkImportState.COPY_FILES);
-    VolumeManager fs = manager.getVolumeManager();
-
-    if (!fs.exists(new Path(error, BulkImport.FAILURES_TXT))) {
-      return new CleanUpBulkImport(tableId, source, bulk, error);
-    }
-
-    var failures = new HashSet<Path>();
-    var loadedFailures = new HashSet<Path>();
-
-    try (BufferedReader in = new BufferedReader(
-        new InputStreamReader(fs.open(new Path(error, BulkImport.FAILURES_TXT)), UTF_8))) {
-      String line = null;
-      while ((line = in.readLine()) != null) {
-        Path path = new Path(line);
-        if (!fs.exists(new Path(error, path.getName()))) {
-          failures.add(path);
-        }
-      }
-    }
-
-    /*
-     * I thought I could move files that have no file references in the table. However its possible
-     * a clone references a file. Therefore only move files that have no loaded markers.
-     */
-
-    // determine which failed files were loaded
-    AccumuloClient client = manager.getContext();
-    try (Scanner mscanner =
-        new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY))) {
-      mscanner.setRange(new KeyExtent(tableId, null, null).toMetaRange());
-      mscanner.fetchColumnFamily(BulkFileColumnFamily.NAME);
-
-      for (Entry<Key,Value> entry : mscanner) {
-        if (BulkFileColumnFamily.getBulkLoadTid(entry.getValue()) == tid) {
-          Path loadedFile =
-              new Path(ValidationUtil.validate(entry.getKey().getColumnQualifierData().toString()));
-          if (failures.remove(loadedFile)) {
-            loadedFailures.add(loadedFile);
-          }
-        }
-      }
-    }
-
-    // move failed files that were not loaded
-    for (Path orig : failures) {
-      Path dest = new Path(error, orig.getName());
-      fs.rename(orig, dest);
-      log.debug(FateTxId.formatTid(tid) + " renamed " + orig + " to " + dest + ": import failed");
-    }
-
-    if (!loadedFailures.isEmpty()) {
-      DistributedWorkQueue bifCopyQueue = new DistributedWorkQueue(
-          Constants.ZROOT + "/" + manager.getInstanceID() + Constants.ZBULK_FAILED_COPYQ,
-          manager.getConfiguration(), manager.getContext());
-
-      HashSet<String> workIds = new HashSet<>();
-
-      for (Path orig : loadedFailures) {
-        Path dest = new Path(error, orig.getName());
-
-        if (fs.exists(dest)) {
-          continue;
-        }
-
-        bifCopyQueue.addWork(orig.getName(), (orig + "," + dest).getBytes(UTF_8));
-        workIds.add(orig.getName());
-        log.debug(
-            FateTxId.formatTid(tid) + " added to copyq: " + orig + " to " + dest + ": failed");
-      }
-
-      bifCopyQueue.waitUntilDone(workIds);
-    }
-
-    fs.deleteRecursively(new Path(error, BulkImport.FAILURES_TXT));
-    return new CleanUpBulkImport(tableId, source, bulk, error);
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
deleted file mode 100644
index 972ec3f..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
+++ /dev/null
@@ -1,238 +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.manager.tableOps.bulkVer1;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.BufferedWriter;
-import java.io.OutputStreamWriter;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
-
-import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException;
-import org.apache.accumulo.core.clientImpl.thrift.ClientService;
-import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
-import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.FateTxId;
-import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-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.accumulo.manager.Manager;
-import org.apache.accumulo.manager.tableOps.ManagerRepo;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-class LoadFiles extends ManagerRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private static final SecureRandom random = new SecureRandom();
-
-  private static ExecutorService threadPool = null;
-  private static final Logger log = LoggerFactory.getLogger(LoadFiles.class);
-
-  private TableId tableId;
-  private String source;
-  private String bulk;
-  private String errorDir;
-  private boolean setTime;
-
-  public LoadFiles(TableId tableId, String source, String bulk, String errorDir, boolean setTime) {
-    this.tableId = tableId;
-    this.source = source;
-    this.bulk = bulk;
-    this.errorDir = errorDir;
-    this.setTime = setTime;
-  }
-
-  @Override
-  public long isReady(long tid, Manager manager) {
-    if (manager.onlineTabletServers().isEmpty()) {
-      return 500;
-    }
-    return 0;
-  }
-
-  private static synchronized ExecutorService getThreadPool(Manager manager) {
-    if (threadPool == null) {
-      threadPool = ThreadPools.getServerThreadPools().createExecutorService(
-          manager.getConfiguration(), Property.MANAGER_BULK_THREADPOOL_SIZE, true);
-    }
-    return threadPool;
-  }
-
-  @Override
-  public Repo<Manager> call(final long tid, final Manager manager) throws Exception {
-    manager.updateBulkImportStatus(source, BulkImportState.LOADING);
-    ExecutorService executor = getThreadPool(manager);
-    final AccumuloConfiguration conf = manager.getConfiguration();
-    VolumeManager fs = manager.getVolumeManager();
-    List<FileStatus> files = new ArrayList<>();
-    Collections.addAll(files, fs.listStatus(new Path(bulk)));
-    log.debug(FateTxId.formatTid(tid) + " importing " + files.size() + " files");
-
-    Path writable = new Path(this.errorDir, ".iswritable");
-    if (!fs.createNewFile(writable)) {
-      // Maybe this is a re-try... clear the flag and try again
-      fs.delete(writable);
-      if (!fs.createNewFile(writable)) {
-        throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
-            TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
-            "Unable to write to " + this.errorDir);
-      }
-    }
-    fs.delete(writable);
-
-    final Set<String> filesToLoad = Collections.synchronizedSet(new HashSet<>());
-    for (FileStatus f : files) {
-      filesToLoad.add(f.getPath().toString());
-    }
-
-    final int RETRIES = Math.max(1, conf.getCount(Property.MANAGER_BULK_RETRIES));
-    for (int attempt = 0; attempt < RETRIES && !filesToLoad.isEmpty(); attempt++) {
-      List<Future<Void>> results = new ArrayList<>();
-
-      if (manager.onlineTabletServers().isEmpty()) {
-        log.warn("There are no tablet server to process bulk import, waiting (tid = "
-            + FateTxId.formatTid(tid) + ")");
-      }
-
-      while (manager.onlineTabletServers().isEmpty()) {
-        sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
-      }
-
-      // Use the threadpool to assign files one-at-a-time to the server
-      final List<String> loaded = Collections.synchronizedList(new ArrayList<>());
-      final TServerInstance[] servers;
-      String prop = conf.get(Property.MANAGER_BULK_TSERVER_REGEX);
-      if (prop == null || "".equals(prop)) {
-        servers = manager.onlineTabletServers().toArray(new TServerInstance[0]);
-      } else {
-        Pattern regex = Pattern.compile(prop);
-        List<TServerInstance> subset = new ArrayList<>();
-        manager.onlineTabletServers().forEach(t -> {
-          if (regex.matcher(t.getHost()).matches()) {
-            subset.add(t);
-          }
-        });
-        if (subset.isEmpty()) {
-          log.warn("There are no tablet servers online that match supplied regex: {}",
-              conf.get(Property.MANAGER_BULK_TSERVER_REGEX));
-        }
-        servers = subset.toArray(new TServerInstance[0]);
-      }
-      if (servers.length > 0) {
-        for (final String file : filesToLoad) {
-          results.add(executor.submit(() -> {
-            ClientService.Client client = null;
-            HostAndPort server = null;
-            try {
-              // get a connection to a random tablet server, do not prefer cached connections
-              // because this is running on the manager and there are lots of connections to tablet
-              // servers serving the metadata tablets
-              long timeInMillis =
-                  manager.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
-              server = servers[random.nextInt(servers.length)].getHostAndPort();
-              client = ThriftUtil.getClient(ThriftClientTypes.CLIENT, server, manager.getContext(),
-                  timeInMillis);
-              List<String> attempt1 = Collections.singletonList(file);
-              log.debug("Asking " + server + " to bulk import " + file);
-              List<String> fail =
-                  client.bulkImportFiles(TraceUtil.traceInfo(), manager.getContext().rpcCreds(),
-                      tid, tableId.canonical(), attempt1, errorDir, setTime);
-              if (fail.isEmpty()) {
-                loaded.add(file);
-              }
-            } catch (Exception ex) {
-              log.error(
-                  "rpc failed server:" + server + ", tid:" + FateTxId.formatTid(tid) + " " + ex);
-            } finally {
-              ThriftUtil.returnClient(client, manager.getContext());
-            }
-            return null;
-          }));
-        }
-      }
-      for (Future<Void> f : results) {
-        f.get();
-      }
-      filesToLoad.removeAll(loaded);
-      if (!filesToLoad.isEmpty()) {
-        log.debug(FateTxId.formatTid(tid) + " attempt " + (attempt + 1) + " "
-            + sampleList(filesToLoad, 10) + " failed");
-        sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
-      }
-    }
-
-    FSDataOutputStream failFile = fs.overwrite(new Path(errorDir, BulkImport.FAILURES_TXT));
-    try (BufferedWriter out = new BufferedWriter(new OutputStreamWriter(failFile, UTF_8))) {
-      for (String f : filesToLoad) {
-        out.write(f);
-        out.write("\n");
-      }
-    }
-
-    // return the next step, which will perform cleanup
-    return new CompleteBulkImport(tableId, source, bulk, errorDir);
-  }
-
-  static String sampleList(Collection<?> potentiallyLongList, int max) {
-    StringBuilder result = new StringBuilder();
-    result.append("[");
-    int i = 0;
-    for (Object obj : potentiallyLongList) {
-      result.append(obj);
-      if (i >= max) {
-        result.append("...");
-        break;
-      } else {
-        result.append(", ");
-      }
-      i++;
-    }
-    if (i < max) {
-      result.delete(result.length() - 2, result.length());
-    }
-    result.append("]");
-    return result.toString();
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java
index 5ace9cc..69378b0 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java
@@ -32,7 +32,7 @@
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.Repo;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -105,9 +105,7 @@
     manager.getContext().getAmple().addBulkLoadInProgressFlag(
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName(), tid);
     AccumuloConfiguration aConf = manager.getConfiguration();
-    @SuppressWarnings("deprecation")
-    int workerCount = aConf.getCount(
-        aConf.resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
+    int workerCount = aConf.getCount(Property.MANAGER_RENAME_THREADS);
     Map<Path,Path> oldToNewMap = new HashMap<>();
     String fmtTid = FateTxId.formatTid(tid);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java
index 12bbacf..5624237 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java
@@ -26,7 +26,7 @@
 import org.apache.accumulo.core.fate.Repo;
 import org.apache.accumulo.core.gc.ReferenceFile;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -59,7 +59,7 @@
     ample.removeBulkLoadInProgressFlag(
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
     ample.putGcFileAndDirCandidates(info.tableId,
-        Collections.singleton(ReferenceFile.forFile(info.tableId, bulkDir.toString())));
+        Collections.singleton(ReferenceFile.forFile(info.tableId, bulkDir)));
     if (info.tableState == TableState.ONLINE) {
 
       Text firstSplit = info.firstSplit == null ? null : new Text(info.firstSplit);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
index d889bbd..8b7885b 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
@@ -31,6 +31,7 @@
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
@@ -42,14 +43,14 @@
 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.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.Repo;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
@@ -57,9 +58,9 @@
 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.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.accumulo.core.util.TextUtil;
@@ -73,6 +74,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 /**
  * Make asynchronous load calls to each overlapping Tablet. This RepO does its work on the isReady
@@ -145,7 +147,7 @@
     // Each RPC to a tablet server needs to check in zookeeper to see if the transaction is still
     // active. The purpose of this map is to group load request by tablet servers inorder to do less
     // RPCs. Less RPCs will result in less calls to Zookeeper.
-    Map<HostAndPort,Map<TKeyExtent,Map<String,MapFileInfo>>> loadQueue;
+    Map<HostAndPort,Map<TKeyExtent,Map<String,DataFileInfo>>> loadQueue;
     private int queuedDataSize = 0;
 
     @Override
@@ -169,9 +171,9 @@
                 tabletFiles.values().stream().mapToInt(Map::size).sum(), tabletFiles.size());
           }
 
-          TabletClientService.Client client = null;
+          Client client = null;
           try {
-            client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server,
+            client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, server,
                 manager.getContext(), timeInMillis);
             client.loadFiles(TraceUtil.traceInfo(), manager.getContext().rpcCreds(), tid,
                 bulkDir.toString(), tabletFiles, setTime);
@@ -188,11 +190,11 @@
     }
 
     private void addToQueue(HostAndPort server, KeyExtent extent,
-        Map<String,MapFileInfo> thriftImports) {
+        Map<String,DataFileInfo> thriftImports) {
       if (!thriftImports.isEmpty()) {
         loadMsgs.increment(server, 1);
 
-        Map<String,MapFileInfo> prev = loadQueue.computeIfAbsent(server, k -> new HashMap<>())
+        Map<String,DataFileInfo> prev = loadQueue.computeIfAbsent(server, k -> new HashMap<>())
             .putIfAbsent(extent.toThrift(), thriftImports);
 
         Preconditions.checkState(prev == null, "Unexpectedly saw extent %s twice", extent);
@@ -219,16 +221,17 @@
           server = location.getHostAndPort();
         }
 
-        Set<TabletFile> loadedFiles = tablet.getLoaded().keySet();
+        Set<ReferencedTabletFile> loadedFiles = tablet.getLoaded().keySet().stream()
+            .map(StoredTabletFile::getTabletFile).collect(Collectors.toSet());
 
-        Map<String,MapFileInfo> thriftImports = new HashMap<>();
+        Map<String,DataFileInfo> thriftImports = new HashMap<>();
 
         for (final Bulk.FileInfo fileInfo : files) {
           Path fullPath = new Path(bulkDir, fileInfo.getFileName());
-          TabletFile bulkFile = new TabletFile(fullPath);
+          ReferencedTabletFile bulkFile = new ReferencedTabletFile(fullPath);
 
           if (!loadedFiles.contains(bulkFile)) {
-            thriftImports.put(fileInfo.getFileName(), new MapFileInfo(fileInfo.getEstFileSize()));
+            thriftImports.put(fileInfo.getFileName(), new DataFileInfo(fileInfo.getEstFileSize()));
           }
         }
 
@@ -270,14 +273,13 @@
     void start(Path bulkDir, Manager manager, long tid, boolean setTime) throws Exception {
       Preconditions.checkArgument(!setTime);
       super.start(bulkDir, manager, tid, setTime);
-      bw = manager.getContext().createBatchWriter(MetadataTable.NAME);
+      bw = manager.getContext().createBatchWriter(AccumuloTable.METADATA.tableName());
       unloadingTablets = new MapCounter<>();
     }
 
     @Override
     void load(List<TabletMetadata> tablets, Files files) throws MutationsRejectedException {
       byte[] fam = TextUtil.getBytes(DataFileColumnFamily.NAME);
-
       for (TabletMetadata tablet : tablets) {
         if (tablet.getLocation() != null) {
           unloadingTablets.increment(tablet.getLocation().getHostAndPort(), 1L);
@@ -287,10 +289,11 @@
         Mutation mutation = new Mutation(tablet.getExtent().toMetaRow());
 
         for (final Bulk.FileInfo fileInfo : files) {
-          String fullPath = new Path(bulkDir, fileInfo.getFileName()).toString();
+          StoredTabletFile fullPath =
+              StoredTabletFile.of(new Path(bulkDir, fileInfo.getFileName()));
           byte[] val =
               new DataFileValue(fileInfo.getEstFileSize(), fileInfo.getEstNumEntries()).encode();
-          mutation.put(fam, fullPath.getBytes(UTF_8), val);
+          mutation.put(fam, fullPath.getMetadata().getBytes(UTF_8), val);
         }
 
         bw.addMutation(mutation);
@@ -325,24 +328,25 @@
 
     Text startRow = loadMapEntry.getKey().prevEndRow();
 
-    Iterator<TabletMetadata> tabletIter =
-        TabletsMetadata.builder(manager.getContext()).forTable(tableId).overlapping(startRow, null)
-            .checkConsistency().fetch(PREV_ROW, LOCATION, LOADED).build().iterator();
-
     Loader loader;
     if (bulkInfo.tableState == TableState.ONLINE) {
       loader = new OnlineLoader();
     } else {
       loader = new OfflineLoader();
     }
-
+    long t1;
     loader.start(bulkDir, manager, tid, bulkInfo.setTime);
+    try (TabletsMetadata tabletsMetadata =
+        TabletsMetadata.builder(manager.getContext()).forTable(tableId).overlapping(startRow, null)
+            .checkConsistency().fetch(PREV_ROW, LOCATION, LOADED).build()) {
 
-    long t1 = System.currentTimeMillis();
-    while (lmi.hasNext()) {
-      loadMapEntry = lmi.next();
-      List<TabletMetadata> tablets = findOverlappingTablets(loadMapEntry.getKey(), tabletIter);
-      loader.load(tablets, loadMapEntry.getValue());
+      t1 = System.currentTimeMillis();
+      while (lmi.hasNext()) {
+        loadMapEntry = lmi.next();
+        List<TabletMetadata> tablets =
+            findOverlappingTablets(loadMapEntry.getKey(), tabletsMetadata.iterator());
+        loader.load(tablets, loadMapEntry.getValue());
+      }
     }
 
     long sleepTime = loader.finish();
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java
index 5611636..e66796c 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java
@@ -18,10 +18,11 @@
  */
 package org.apache.accumulo.manager.tableOps.bulkVer2;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 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.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -44,6 +45,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.file.FilePrefix;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.manager.Manager;
@@ -96,13 +98,12 @@
     if (manager.onlineTabletServers().isEmpty()) {
       return 500;
     }
-    manager.getContext().clearTableListCache();
 
     return Utils.reserveHdfsDirectory(manager, bulkInfo.sourceDir, tid);
   }
 
   @VisibleForTesting
-  interface TabletIterFactory {
+  interface TabletIterFactory extends AutoCloseable {
     Iterator<KeyExtent> newTabletIter(Text startRow);
   }
 
@@ -194,6 +195,33 @@
     return new KeyExtent(firstTablet.tableId(), lastTablet.endRow(), firstTablet.prevEndRow());
   }
 
+  private static class TabletIterFactoryImpl implements TabletIterFactory {
+    private final List<AutoCloseable> resourcesToClose = new ArrayList<>();
+    private final Manager manager;
+    private final BulkInfo bulkInfo;
+
+    public TabletIterFactoryImpl(Manager manager, BulkInfo bulkInfo) {
+      this.manager = manager;
+      this.bulkInfo = bulkInfo;
+    }
+
+    @Override
+    public Iterator<KeyExtent> newTabletIter(Text startRow) {
+      TabletsMetadata tabletsMetadata =
+          TabletsMetadata.builder(manager.getContext()).forTable(bulkInfo.tableId)
+              .overlapping(startRow, null).checkConsistency().fetch(PREV_ROW).build();
+      resourcesToClose.add(tabletsMetadata);
+      return tabletsMetadata.stream().map(TabletMetadata::getExtent).iterator();
+    }
+
+    @Override
+    public void close() throws Exception {
+      for (AutoCloseable resource : resourcesToClose) {
+        resource.close();
+      }
+    }
+  }
+
   private KeyExtent checkForMerge(final long tid, final Manager manager) throws Exception {
 
     VolumeManager fs = manager.getVolumeManager();
@@ -202,14 +230,10 @@
     int maxTablets = manager.getContext().getTableConfiguration(bulkInfo.tableId)
         .getCount(Property.TABLE_BULK_MAX_TABLETS);
 
-    try (LoadMappingIterator lmi =
-        BulkSerialize.readLoadMapping(bulkDir.toString(), bulkInfo.tableId, fs::open)) {
-
-      TabletIterFactory tabletIterFactory =
-          startRow -> TabletsMetadata.builder(manager.getContext()).forTable(bulkInfo.tableId)
-              .overlapping(startRow, null).checkConsistency().fetch(PREV_ROW).build().stream()
-              .map(TabletMetadata::getExtent).iterator();
-
+    try (
+        LoadMappingIterator lmi =
+            BulkSerialize.readLoadMapping(bulkDir.toString(), bulkInfo.tableId, fs::open);
+        TabletIterFactory tabletIterFactory = new TabletIterFactoryImpl(manager, bulkInfo)) {
       return validateLoadMapping(bulkInfo.tableId.canonical(), lmi, tabletIterFactory, maxTablets,
           tid);
     }
@@ -242,8 +266,8 @@
 
     for (FileStatus file : files) {
       // since these are only valid files we know it has an extension
-      String newName =
-          "I" + namer.getNextName() + "." + FilenameUtils.getExtension(file.getPath().getName());
+      String newName = FilePrefix.BULK_IMPORT.toPrefix() + namer.getNextName() + "."
+          + FilenameUtils.getExtension(file.getPath().getName());
       oldToNewNameMap.put(file.getPath().getName(), new Path(bulkDir, newName).getName());
     }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneTable.java
index 352f3d1..0eb2368 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneTable.java
@@ -25,6 +25,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -48,9 +49,9 @@
 
   @Override
   public long isReady(long tid, Manager environment) throws Exception {
-    long val = Utils.reserveNamespace(environment, cloneInfo.srcNamespaceId, tid, false, true,
-        TableOperation.CLONE);
-    val += Utils.reserveTable(environment, cloneInfo.srcTableId, tid, false, true,
+    long val = Utils.reserveNamespace(environment, cloneInfo.srcNamespaceId, tid, LockType.READ,
+        true, TableOperation.CLONE);
+    val += Utils.reserveTable(environment, cloneInfo.srcTableId, tid, LockType.READ, true,
         TableOperation.CLONE);
     return val;
   }
@@ -71,8 +72,8 @@
 
   @Override
   public void undo(long tid, Manager environment) {
-    Utils.unreserveNamespace(environment, cloneInfo.srcNamespaceId, tid, false);
-    Utils.unreserveTable(environment, cloneInfo.srcTableId, tid, false);
+    Utils.unreserveNamespace(environment, cloneInfo.srcNamespaceId, tid, LockType.READ);
+    Utils.unreserveTable(environment, cloneInfo.srcTableId, tid, LockType.READ);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneZookeeper.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneZookeeper.java
index cfc10d0..7cfdf31 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneZookeeper.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneZookeeper.java
@@ -23,6 +23,7 @@
 import org.apache.accumulo.core.clientImpl.Namespaces;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.util.tables.TableNameUtil;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -45,11 +46,11 @@
   public long isReady(long tid, Manager environment) throws Exception {
     long val = 0;
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId)) {
-      val += Utils.reserveNamespace(environment, cloneInfo.namespaceId, tid, false, true,
+      val += Utils.reserveNamespace(environment, cloneInfo.namespaceId, tid, LockType.READ, true,
           TableOperation.CLONE);
     }
-    val +=
-        Utils.reserveTable(environment, cloneInfo.tableId, tid, true, false, TableOperation.CLONE);
+    val += Utils.reserveTable(environment, cloneInfo.tableId, tid, LockType.WRITE, false,
+        TableOperation.CLONE);
     return val;
   }
 
@@ -77,9 +78,9 @@
   public void undo(long tid, Manager environment) throws Exception {
     environment.getTableManager().removeTable(cloneInfo.tableId);
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId)) {
-      Utils.unreserveNamespace(environment, cloneInfo.namespaceId, tid, false);
+      Utils.unreserveNamespace(environment, cloneInfo.namespaceId, tid, LockType.READ);
     }
-    Utils.unreserveTable(environment, cloneInfo.tableId, tid, true);
+    Utils.unreserveTable(environment, cloneInfo.tableId, tid, LockType.WRITE);
     environment.getContext().clearTableListCache();
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/FinishCloneTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/FinishCloneTable.java
index 9c2b46f..f424e56 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/FinishCloneTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/FinishCloneTable.java
@@ -21,6 +21,7 @@
 import java.util.EnumSet;
 
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -58,12 +59,12 @@
           expectedCurrStates);
     }
 
-    Utils.unreserveNamespace(environment, cloneInfo.srcNamespaceId, tid, false);
+    Utils.unreserveNamespace(environment, cloneInfo.srcNamespaceId, tid, LockType.READ);
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId)) {
-      Utils.unreserveNamespace(environment, cloneInfo.namespaceId, tid, false);
+      Utils.unreserveNamespace(environment, cloneInfo.namespaceId, tid, LockType.READ);
     }
-    Utils.unreserveTable(environment, cloneInfo.srcTableId, tid, false);
-    Utils.unreserveTable(environment, cloneInfo.tableId, tid, true);
+    Utils.unreserveTable(environment, cloneInfo.srcTableId, tid, LockType.READ);
+    Utils.unreserveTable(environment, cloneInfo.tableId, tid, LockType.WRITE);
 
     environment.getEventCoordinator().event("Cloned table %s from %s", cloneInfo.tableName,
         cloneInfo.srcTableId);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java
index 5f396e0..d172a62 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactRange.java
@@ -27,13 +27,13 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException;
-import org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil;
 import org.apache.accumulo.core.clientImpl.UserCompactionUtils;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.accumulo.core.util.TextUtil;
@@ -66,14 +66,13 @@
     this.namespaceId = namespaceId;
 
     if (!compactionConfig.getIterators().isEmpty()
-        || !CompactionStrategyConfigUtil.isDefault(compactionConfig)
         || !compactionConfig.getExecutionHints().isEmpty()
         || !isDefault(compactionConfig.getConfigurer())
         || !isDefault(compactionConfig.getSelector())) {
       this.config = UserCompactionUtils.encode(compactionConfig);
     } else {
       log.debug(
-          "Using default compaction strategy. No user iterators or compaction strategy provided.");
+          "Using default compaction config. No user iterators or compaction config provided.");
     }
 
     if (compactionConfig.getStartRow() != null && compactionConfig.getEndRow() != null
@@ -91,8 +90,9 @@
 
   @Override
   public long isReady(long tid, Manager env) throws Exception {
-    return Utils.reserveNamespace(env, namespaceId, tid, false, true, TableOperation.COMPACT)
-        + Utils.reserveTable(env, tableId, tid, false, true, TableOperation.COMPACT);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, true,
+        TableOperation.COMPACT)
+        + Utils.reserveTable(env, tableId, tid, LockType.READ, true, TableOperation.COMPACT);
   }
 
   @Override
@@ -181,8 +181,8 @@
     try {
       removeIterators(env, tid, tableId);
     } finally {
-      Utils.unreserveNamespace(env, namespaceId, tid, false);
-      Utils.unreserveTable(env, tableId, tid, false);
+      Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
+      Utils.unreserveTable(env, tableId, tid, LockType.READ);
     }
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java
index b17ce8b..992eba9 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java
@@ -34,7 +34,7 @@
 import org.apache.accumulo.core.fate.Repo;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
@@ -74,7 +74,7 @@
   @Override
   public long isReady(long tid, Manager manager) throws Exception {
 
-    if (tableId.equals(RootTable.ID)) {
+    if (tableId.equals(AccumuloTable.ROOT.tableId())) {
       // this codes not properly handle the root table. See #798
       return 0;
     }
@@ -104,18 +104,19 @@
     int tabletsToWaitFor = 0;
     int tabletCount = 0;
 
-    TabletsMetadata tablets = TabletsMetadata.builder(manager.getContext()).forTable(tableId)
-        .overlapping(startRow, endRow).fetch(LOCATION, PREV_ROW, COMPACT_ID).build();
+    try (TabletsMetadata tablets = TabletsMetadata.builder(manager.getContext()).forTable(tableId)
+        .overlapping(startRow, endRow).fetch(LOCATION, PREV_ROW, COMPACT_ID).build()) {
 
-    for (TabletMetadata tablet : tablets) {
-      if (tablet.getCompactId().orElse(-1) < compactId) {
-        tabletsToWaitFor++;
-        if (tablet.hasCurrent()) {
-          serversToFlush.increment(tablet.getLocation().getServerInstance(), 1);
+      for (TabletMetadata tablet : tablets) {
+        if (tablet.getCompactId().orElse(-1) < compactId) {
+          tabletsToWaitFor++;
+          if (tablet.hasCurrent()) {
+            serversToFlush.increment(tablet.getLocation().getServerInstance(), 1);
+          }
         }
-      }
 
-      tabletCount++;
+        tabletCount++;
+      }
     }
 
     long scanTime = System.currentTimeMillis() - t1;
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java
index 7d25f2d..47d5341 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/CancelCompactions.java
@@ -26,6 +26,7 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.FateTxId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -48,8 +49,9 @@
 
   @Override
   public long isReady(long tid, Manager env) throws Exception {
-    return Utils.reserveNamespace(env, namespaceId, tid, false, true, TableOperation.COMPACT_CANCEL)
-        + Utils.reserveTable(env, tableId, tid, false, true, TableOperation.COMPACT_CANCEL);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, true,
+        TableOperation.COMPACT_CANCEL)
+        + Utils.reserveTable(env, tableId, tid, LockType.READ, true, TableOperation.COMPACT_CANCEL);
   }
 
   @Override
@@ -60,8 +62,8 @@
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveTable(env, tableId, tid, false);
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
+    Utils.unreserveTable(env, tableId, tid, LockType.READ);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
   }
 
   public static void mutateZooKeeper(long tid, TableId tableId, Manager environment)
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/FinishCancelCompaction.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/FinishCancelCompaction.java
index 70fa526..36a7e26 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/FinishCancelCompaction.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/cancel/FinishCancelCompaction.java
@@ -21,6 +21,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -37,8 +38,8 @@
 
   @Override
   public Repo<Manager> call(long tid, Manager environment) {
-    Utils.unreserveTable(environment, tableId, tid, false);
-    Utils.unreserveNamespace(environment, namespaceId, tid, false);
+    Utils.unreserveTable(environment, tableId, tid, LockType.READ);
+    Utils.unreserveNamespace(environment, namespaceId, tid, LockType.READ);
     return null;
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java
index f1f71a8..9d0d653 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java
@@ -27,6 +27,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.TableInfo;
@@ -60,7 +61,7 @@
   @Override
   public long isReady(long tid, Manager environment) throws Exception {
     // reserve the table's namespace to make sure it doesn't change while the table is created
-    return Utils.reserveNamespace(environment, tableInfo.getNamespaceId(), tid, false, true,
+    return Utils.reserveNamespace(environment, tableInfo.getNamespaceId(), tid, LockType.READ, true,
         TableOperation.CREATE);
   }
 
@@ -95,7 +96,7 @@
     } catch (IOException e) {
       log.error("Table failed to be created and failed to clean up split files at {}", p, e);
     } finally {
-      Utils.unreserveNamespace(env, tableInfo.getNamespaceId(), tid, false);
+      Utils.unreserveNamespace(env, tableInfo.getNamespaceId(), tid, LockType.READ);
     }
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/FinishCreateTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/FinishCreateTable.java
index 519ef36..51dcdce 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/FinishCreateTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/FinishCreateTable.java
@@ -23,6 +23,7 @@
 
 import org.apache.accumulo.core.client.admin.InitialTableState;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -61,8 +62,8 @@
           TableState.ONLINE, expectedCurrStates);
     }
 
-    Utils.unreserveNamespace(env, tableInfo.getNamespaceId(), tid, false);
-    Utils.unreserveTable(env, tableInfo.getTableId(), tid, true);
+    Utils.unreserveNamespace(env, tableInfo.getNamespaceId(), tid, LockType.READ);
+    Utils.unreserveTable(env, tableInfo.getTableId(), tid, LockType.WRITE);
 
     env.getEventCoordinator().event("Created table %s ", tableInfo.getTableName());
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java
index cbc65a8..a89858b 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java
@@ -32,8 +32,8 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataTime;
@@ -72,7 +72,8 @@
       SortedSet<Text> splits = Utils.getSortedSetFromFile(env, tableInfo.getSplitPath(), true);
       SortedSet<Text> dirs = Utils.getSortedSetFromFile(env, tableInfo.getSplitDirsPath(), false);
       Map<Text,Text> splitDirMap = createSplitDirectoryMap(splits, dirs);
-      try (BatchWriter bw = env.getContext().createBatchWriter(MetadataTable.NAME)) {
+      try (
+          BatchWriter bw = env.getContext().createBatchWriter(AccumuloTable.METADATA.tableName())) {
         writeSplitsToMetadataTable(env.getContext(), tableInfo.getTableId(), splits, splitDirMap,
             tableInfo.getTimeType(), env.getManagerLock(), bw);
       }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
index 40ad2d2..1bb6963 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
@@ -22,6 +22,7 @@
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.TableInfo;
@@ -41,7 +42,7 @@
 
   @Override
   public long isReady(long tid, Manager environment) throws Exception {
-    return Utils.reserveTable(environment, tableInfo.getTableId(), tid, true, false,
+    return Utils.reserveTable(environment, tableInfo.getTableId(), tid, LockType.WRITE, false,
         TableOperation.CREATE);
   }
 
@@ -79,7 +80,7 @@
   @Override
   public void undo(long tid, Manager manager) throws Exception {
     manager.getTableManager().removeTable(tableInfo.getTableId());
-    Utils.unreserveTable(manager, tableInfo.getTableId(), tid, true);
+    Utils.unreserveTable(manager, tableInfo.getTableId(), tid, LockType.WRITE);
     manager.getContext().clearTableListCache();
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
index 038db0b..6b59a39 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
@@ -35,8 +35,9 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.iterators.user.GrepIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletState;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -90,7 +91,8 @@
 
     boolean done = true;
     Range tableRange = new KeyExtent(tableId, null, null).toMetaRange();
-    Scanner scanner = manager.getContext().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner scanner = manager.getContext().createScanner(AccumuloTable.METADATA.tableName(),
+        Authorizations.EMPTY);
     MetaDataTableScanner.configureScanner(scanner, manager);
     scanner.setRange(tableRange);
 
@@ -126,7 +128,7 @@
       // look for other tables that references this table's files
       AccumuloClient client = manager.getContext();
       try (BatchScanner bs =
-          client.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 8)) {
+          client.createBatchScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY, 8)) {
         Range allTables = TabletsSection.getRange();
         Range tableRange = TabletsSection.getRange(tableId);
         Range beforeTable =
@@ -147,8 +149,8 @@
 
     } catch (Exception e) {
       refCount = -1;
-      log.error("Failed to scan " + MetadataTable.NAME + " looking for references to deleted table "
-          + tableId, e);
+      log.error("Failed to scan " + AccumuloTable.METADATA.tableName()
+          + " looking for references to deleted table " + tableId, e);
     }
 
     // remove metadata table entries
@@ -171,7 +173,7 @@
     }
 
     if (refCount == 0) {
-      // delete the map files
+      // delete the data files
       try {
         VolumeManager fs = manager.getVolumeManager();
         for (String dir : manager.getContext().getTablesDirs()) {
@@ -205,8 +207,8 @@
       log.error("{}", e.getMessage(), e);
     }
 
-    Utils.unreserveTable(manager, tableId, tid, true);
-    Utils.unreserveNamespace(manager, namespaceId, tid, false);
+    Utils.unreserveTable(manager, tableId, tid, LockType.WRITE);
+    Utils.unreserveNamespace(manager, namespaceId, tid, LockType.READ);
 
     LoggerFactory.getLogger(CleanUp.class).debug("Deleted table " + tableId);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/DeleteTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/DeleteTable.java
index fcd0e79..ae325d4 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/DeleteTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/DeleteTable.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -43,8 +44,9 @@
 
   @Override
   public long isReady(long tid, Manager env) throws Exception {
-    return Utils.reserveNamespace(env, namespaceId, tid, false, false, TableOperation.DELETE)
-        + Utils.reserveTable(env, tableId, tid, true, true, TableOperation.DELETE);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, false,
+        TableOperation.DELETE)
+        + Utils.reserveTable(env, tableId, tid, LockType.WRITE, true, TableOperation.DELETE);
   }
 
   @Override
@@ -58,7 +60,7 @@
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveTable(env, tableId, tid, true);
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
+    Utils.unreserveTable(env, tableId, tid, LockType.WRITE);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java
index 7ba0700..e00fe99 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/PreDeleteTable.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.manager.Manager;
@@ -51,8 +52,8 @@
 
   @Override
   public long isReady(long tid, Manager env) throws Exception {
-    return Utils.reserveNamespace(env, namespaceId, tid, false, true, TableOperation.DELETE)
-        + Utils.reserveTable(env, tableId, tid, false, true, TableOperation.DELETE);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, true, TableOperation.DELETE)
+        + Utils.reserveTable(env, tableId, tid, LockType.READ, true, TableOperation.DELETE);
   }
 
   private void preventFutureCompactions(Manager environment)
@@ -69,15 +70,15 @@
       CancelCompactions.mutateZooKeeper(tid, tableId, environment);
       return new DeleteTable(namespaceId, tableId);
     } finally {
-      Utils.unreserveTable(environment, tableId, tid, false);
-      Utils.unreserveNamespace(environment, namespaceId, tid, false);
+      Utils.unreserveTable(environment, tableId, tid, LockType.READ);
+      Utils.unreserveNamespace(environment, namespaceId, tid, LockType.READ);
     }
   }
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveTable(env, tableId, tid, false);
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
+    Utils.unreserveTable(env, tableId, tid, LockType.READ);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java
index 2990b9e..26f7589 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java
@@ -25,7 +25,8 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -50,8 +51,8 @@
 
   @Override
   public long isReady(long tid, Manager env) throws Exception {
-    return Utils.reserveNamespace(env, namespaceId, tid, false, true, TableOperation.MERGE)
-        + Utils.reserveTable(env, tableId, tid, true, true, TableOperation.MERGE);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, true, TableOperation.MERGE)
+        + Utils.reserveTable(env, tableId, tid, LockType.WRITE, true, TableOperation.MERGE);
   }
 
   public TableRangeOp(MergeInfo.Operation op, NamespaceId namespaceId, TableId tableId,
@@ -66,9 +67,9 @@
   @Override
   public Repo<Manager> call(long tid, Manager env) throws Exception {
 
-    if (RootTable.ID.equals(tableId) && Operation.MERGE.equals(op)) {
+    if (AccumuloTable.ROOT.tableId().equals(tableId) && Operation.MERGE.equals(op)) {
       log.warn("Attempt to merge tablets for {} does nothing. It is not splittable.",
-          RootTable.NAME);
+          AccumuloTable.ROOT.tableName());
     }
 
     Text start = startRow.length == 0 ? null : new Text(startRow);
@@ -102,8 +103,8 @@
       log.info("removing merge information {}", mergeInfo);
     }
     env.clearMergeState(tableId);
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
-    Utils.unreserveTable(env, tableId, tid, true);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
+    Utils.unreserveTable(env, tableId, tid, LockType.WRITE);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java
index 9b73d4c..4597ba6 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java
@@ -21,6 +21,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -71,8 +72,8 @@
     MergeInfo mergeInfo = manager.getMergeInfo(tableId);
     log.info("removing merge information " + mergeInfo);
     manager.clearMergeState(tableId);
-    Utils.unreserveTable(manager, tableId, tid, true);
-    Utils.unreserveNamespace(manager, namespaceId, tid, false);
+    Utils.unreserveTable(manager, tableId, tid, LockType.WRITE);
+    Utils.unreserveNamespace(manager, namespaceId, tid, LockType.READ);
     return null;
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/FinishCreateNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/FinishCreateNamespace.java
index bb462f9..0a5897d 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/FinishCreateNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/FinishCreateNamespace.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.manager.tableOps.namespace.create;
 
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -42,7 +43,7 @@
   @Override
   public Repo<Manager> call(long id, Manager env) {
 
-    Utils.unreserveNamespace(env, namespaceInfo.namespaceId, id, true);
+    Utils.unreserveNamespace(env, namespaceInfo.namespaceId, id, LockType.WRITE);
 
     env.getEventCoordinator().event("Created namespace %s ", namespaceInfo.namespaceName);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
index 1b054b2..939b6f0 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
@@ -20,6 +20,7 @@
 
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -40,7 +41,7 @@
 
   @Override
   public long isReady(long id, Manager environment) throws Exception {
-    return Utils.reserveNamespace(environment, namespaceInfo.namespaceId, id, true, false,
+    return Utils.reserveNamespace(environment, namespaceInfo.namespaceId, id, LockType.WRITE, false,
         TableOperation.CREATE);
   }
 
@@ -71,7 +72,7 @@
   public void undo(long tid, Manager manager) throws Exception {
     manager.getTableManager().removeNamespace(namespaceInfo.namespaceId);
     manager.getContext().clearTableListCache();
-    Utils.unreserveNamespace(manager, namespaceInfo.namespaceId, tid, true);
+    Utils.unreserveNamespace(manager, namespaceInfo.namespaceId, tid, LockType.WRITE);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/DeleteNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/DeleteNamespace.java
index 6dd61c5..e9a8b49 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/DeleteNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/DeleteNamespace.java
@@ -21,6 +21,7 @@
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -37,7 +38,8 @@
 
   @Override
   public long isReady(long id, Manager environment) throws Exception {
-    return Utils.reserveNamespace(environment, namespaceId, id, true, true, TableOperation.DELETE);
+    return Utils.reserveNamespace(environment, namespaceId, id, LockType.WRITE, true,
+        TableOperation.DELETE);
   }
 
   @Override
@@ -48,7 +50,7 @@
 
   @Override
   public void undo(long id, Manager environment) {
-    Utils.unreserveNamespace(environment, namespaceId, id, true);
+    Utils.unreserveNamespace(environment, namespaceId, id, LockType.WRITE);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java
index 27fc850..db4c9a5 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java
@@ -21,6 +21,7 @@
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -63,7 +64,7 @@
       log.error("{}", e.getMessage(), e);
     }
 
-    Utils.unreserveNamespace(manager, namespaceId, id, true);
+    Utils.unreserveNamespace(manager, namespaceId, id, LockType.WRITE);
 
     log.debug("Deleted namespace " + namespaceId);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java
index 4eabcde..a217f0c 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java
@@ -26,6 +26,7 @@
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -41,7 +42,8 @@
 
   @Override
   public long isReady(long id, Manager environment) throws Exception {
-    return Utils.reserveNamespace(environment, namespaceId, id, true, true, TableOperation.RENAME);
+    return Utils.reserveNamespace(environment, namespaceId, id, LockType.WRITE, true,
+        TableOperation.RENAME);
   }
 
   public RenameNamespace(NamespaceId namespaceId, String oldName, String newName) {
@@ -77,7 +79,7 @@
       manager.getContext().clearTableListCache();
     } finally {
       Utils.getTableNameLock().unlock();
-      Utils.unreserveNamespace(manager, namespaceId, id, true);
+      Utils.unreserveNamespace(manager, namespaceId, id, LockType.WRITE);
     }
 
     LoggerFactory.getLogger(RenameNamespace.class).debug("Renamed namespace {} {} {}", namespaceId,
@@ -88,7 +90,7 @@
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveNamespace(env, namespaceId, tid, true);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.WRITE);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java
index a91b0ca..ac6f53a 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java
@@ -29,6 +29,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.tables.TableNameUtil;
@@ -47,8 +48,8 @@
 
   @Override
   public long isReady(long tid, Manager env) throws Exception {
-    return Utils.reserveNamespace(env, namespaceId, tid, false, true, TableOperation.RENAME)
-        + Utils.reserveTable(env, tableId, tid, true, true, TableOperation.RENAME);
+    return Utils.reserveNamespace(env, namespaceId, tid, LockType.READ, true, TableOperation.RENAME)
+        + Utils.reserveTable(env, tableId, tid, LockType.WRITE, true, TableOperation.RENAME);
   }
 
   public RenameTable(NamespaceId namespaceId, TableId tableId, String oldTableName,
@@ -100,8 +101,8 @@
       manager.getContext().clearTableListCache();
     } finally {
       Utils.getTableNameLock().unlock();
-      Utils.unreserveTable(manager, tableId, tid, true);
-      Utils.unreserveNamespace(manager, namespaceId, tid, false);
+      Utils.unreserveTable(manager, tableId, tid, LockType.WRITE);
+      Utils.unreserveNamespace(manager, namespaceId, tid, LockType.READ);
     }
 
     LoggerFactory.getLogger(RenameTable.class).debug("Renamed table {} {} {}", tableId,
@@ -112,8 +113,8 @@
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveTable(env, tableId, tid, true);
-    Utils.unreserveNamespace(env, namespaceId, tid, false);
+    Utils.unreserveTable(env, tableId, tid, LockType.WRITE);
+    Utils.unreserveNamespace(env, namespaceId, tid, LockType.READ);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/ExportTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/ExportTable.java
index 7e3a2a3..8662607 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/ExportTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/ExportTable.java
@@ -54,7 +54,15 @@
     Utils.unreserveHdfsDirectory(env, new Path(tableInfo.exportDir).toString(), tid);
   }
 
-  public static final int VERSION = 1;
+  /**
+   * Defines export / version.
+   * <ul>
+   * <li>version 1 exported by Accumulo &lt; 3.1</li>
+   * <li>version 2 exported by Accumulo =&gt; 3.1 - uses file references with ranges.</li>
+   * </ul>
+   */
+  public static final int VERSION_2 = 2;
+  public static final int CURR_VERSION = VERSION_2;
 
   public static final String DATA_VERSION_PROP = "srcDataVersion";
   public static final String EXPORT_VERSION_PROP = "exportVersion";
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java
index a245a95..aaaf1fd 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java
@@ -48,8 +48,10 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.ValidationUtil;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
@@ -91,9 +93,10 @@
   @Override
   public long isReady(long tid, Manager manager) throws Exception {
 
-    long reserved = Utils.reserveNamespace(manager, tableInfo.namespaceID, tid, false, true,
+    long reserved = Utils.reserveNamespace(manager, tableInfo.namespaceID, tid, LockType.READ, true,
         TableOperation.EXPORT)
-        + Utils.reserveTable(manager, tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
+        + Utils.reserveTable(manager, tableInfo.tableID, tid, LockType.READ, true,
+            TableOperation.EXPORT);
     if (reserved > 0) {
       return reserved;
     }
@@ -102,7 +105,8 @@
 
     checkOffline(manager.getContext());
 
-    Scanner metaScanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner metaScanner =
+        client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
     metaScanner.setRange(new KeyExtent(tableInfo.tableID, null, null).toMetaRange());
 
     // scan for locations
@@ -139,16 +143,16 @@
           tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
           "Failed to create export files " + ioe.getMessage());
     }
-    Utils.unreserveNamespace(manager, tableInfo.namespaceID, tid, false);
-    Utils.unreserveTable(manager, tableInfo.tableID, tid, false);
+    Utils.unreserveNamespace(manager, tableInfo.namespaceID, tid, LockType.READ);
+    Utils.unreserveTable(manager, tableInfo.tableID, tid, LockType.READ);
     Utils.unreserveHdfsDirectory(manager, new Path(tableInfo.exportDir).toString(), tid);
     return null;
   }
 
   @Override
   public void undo(long tid, Manager env) {
-    Utils.unreserveNamespace(env, tableInfo.namespaceID, tid, false);
-    Utils.unreserveTable(env, tableInfo.tableID, tid, false);
+    Utils.unreserveNamespace(env, tableInfo.namespaceID, tid, LockType.READ);
+    Utils.unreserveTable(env, tableInfo.tableID, tid, LockType.READ);
   }
 
   public static void exportTable(VolumeManager fs, ServerContext context, String tableName,
@@ -166,7 +170,7 @@
     try (OutputStreamWriter osw = new OutputStreamWriter(dataOut, UTF_8)) {
 
       zipOut.putNextEntry(new ZipEntry(Constants.EXPORT_INFO_FILE));
-      osw.append(ExportTable.EXPORT_VERSION_PROP + ":" + ExportTable.VERSION + "\n");
+      osw.append(ExportTable.EXPORT_VERSION_PROP + ":" + ExportTable.CURR_VERSION + "\n");
       osw.append("srcInstanceName:" + context.getInstanceName() + "\n");
       osw.append("srcInstanceID:" + context.getInstanceID() + "\n");
       osw.append("srcZookeepers:" + context.getZooKeepers() + "\n");
@@ -226,7 +230,8 @@
 
     Map<String,String> uniqueFiles = new HashMap<>();
 
-    Scanner metaScanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner metaScanner =
+        context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
     metaScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
     TabletColumnFamily.PREV_ROW_COLUMN.fetch(metaScanner);
     ServerColumnFamily.TIME_COLUMN.fetch(metaScanner);
@@ -237,7 +242,9 @@
       entry.getValue().write(dataOut);
 
       if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-        String path = ValidationUtil.validate(entry.getKey().getColumnQualifierData().toString());
+        // We need to get the actual path of the file to validate unique files
+        String path = ValidationUtil.validate(StoredTabletFile
+            .of(entry.getKey().getColumnQualifierData().toString()).getMetadataPath());
         String[] tokens = path.split("/");
         if (tokens.length < 1) {
           throw new RuntimeException("Illegal path " + path);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/FinishImportTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/FinishImportTable.java
index 125e463..2421636 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/FinishImportTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/FinishImportTable.java
@@ -23,6 +23,7 @@
 import java.util.EnumSet;
 
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -58,8 +59,8 @@
     final TableState newState = tableInfo.keepOffline ? TableState.OFFLINE : TableState.ONLINE;
     env.getTableManager().transitionTableState(tableInfo.tableId, newState, expectedCurrStates);
 
-    Utils.unreserveNamespace(env, tableInfo.namespaceId, tid, false);
-    Utils.unreserveTable(env, tableInfo.tableId, tid, true);
+    Utils.unreserveNamespace(env, tableInfo.namespaceId, tid, LockType.READ);
+    Utils.unreserveTable(env, tableInfo.tableId, tid, LockType.WRITE);
 
     for (ImportedTableInfo.DirectoryMapping dm : tableInfo.directories) {
       Utils.unreserveHdfsDirectory(env, new Path(dm.exportDir).toString(), tid);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java
index eb57a80..8710ec2 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportPopulateZookeeper.java
@@ -28,6 +28,7 @@
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.util.tables.TableNameUtil;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
@@ -50,7 +51,7 @@
 
   @Override
   public long isReady(long tid, Manager environment) throws Exception {
-    return Utils.reserveTable(environment, tableInfo.tableId, tid, true, false,
+    return Utils.reserveTable(environment, tableInfo.tableId, tid, LockType.WRITE, false,
         TableOperation.IMPORT);
   }
 
@@ -104,7 +105,7 @@
   @Override
   public void undo(long tid, Manager env) throws Exception {
     env.getTableManager().removeTable(tableInfo.tableId);
-    Utils.unreserveTable(env, tableInfo.tableId, tid, true);
+    Utils.unreserveTable(env, tableInfo.tableId, tid, LockType.WRITE);
     env.getContext().clearTableListCache();
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java
index 484db2b..7addca6 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java
@@ -40,6 +40,7 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.Repo;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
@@ -78,7 +79,7 @@
     for (ImportedTableInfo.DirectoryMapping dm : tableInfo.directories) {
       result += Utils.reserveHdfsDirectory(environment, new Path(dm.exportDir).toString(), tid);
     }
-    result += Utils.reserveNamespace(environment, tableInfo.namespaceId, tid, false, true,
+    result += Utils.reserveNamespace(environment, tableInfo.namespaceId, tid, LockType.READ, true,
         TableOperation.IMPORT);
     return result;
   }
@@ -110,7 +111,7 @@
 
     log.debug("Searching for export file in {}", exportDirs);
 
-    Integer exportVersion = null;
+    tableInfo.exportedVersion = null;
     Integer dataVersion = null;
 
     try {
@@ -127,7 +128,7 @@
           while ((line = in.readLine()) != null) {
             String[] sa = line.split(":", 2);
             if (sa[0].equals(ExportTable.EXPORT_VERSION_PROP)) {
-              exportVersion = Integer.parseInt(sa[1]);
+              tableInfo.exportedVersion = Integer.parseInt(sa[1]);
             } else if (sa[0].equals(ExportTable.DATA_VERSION_PROP)) {
               dataVersion = Integer.parseInt(sa[1]);
             }
@@ -142,10 +143,10 @@
           "Failed to read export metadata " + e.getMessage());
     }
 
-    if (exportVersion == null || exportVersion > ExportTable.VERSION) {
+    if (tableInfo.exportedVersion == null || tableInfo.exportedVersion > ExportTable.CURR_VERSION) {
       throw new AcceptableThriftTableOperationException(null, tableInfo.tableName,
           TableOperation.IMPORT, TableOperationExceptionType.OTHER,
-          "Incompatible export version " + exportVersion);
+          "Incompatible export version " + tableInfo.exportedVersion);
     }
 
     if (dataVersion == null || dataVersion > AccumuloDataVersion.get()) {
@@ -161,7 +162,7 @@
       Utils.unreserveHdfsDirectory(env, new Path(dm.exportDir).toString(), tid);
     }
 
-    Utils.unreserveNamespace(env, tableInfo.namespaceId, tid, false);
+    Utils.unreserveNamespace(env, tableInfo.namespaceId, tid, LockType.READ);
   }
 
   static List<ImportedTableInfo.DirectoryMapping> parseExportDir(Set<String> exportDirs) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportedTableInfo.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportedTableInfo.java
index 306c7a3..13ea263 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportedTableInfo.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportedTableInfo.java
@@ -26,7 +26,7 @@
 
 class ImportedTableInfo implements Serializable {
 
-  private static final long serialVersionUID = 1L;
+  private static final long serialVersionUID = 2L;
 
   public String user;
   public String tableName;
@@ -36,6 +36,7 @@
   public String exportFile;
   public boolean keepMappings;
   public boolean keepOffline;
+  public Integer exportedVersion = null;
 
   static class DirectoryMapping implements Serializable {
     private static final long serialVersionUID = 1L;
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java
index 9d23f06..19c9635 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MapImportFileNames.java
@@ -25,7 +25,6 @@
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperation;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
@@ -78,12 +77,13 @@
           if (sa.length > 1) {
             extension = sa[sa.length - 1];
 
+            // skip files with unknown extensions
             if (!FileOperations.getValidExtensions().contains(extension)) {
               continue;
             }
           } else {
-            // assume it is a map file
-            extension = Constants.MAPFILE_EXTENSION;
+            // skip files without an extension
+            continue;
           }
 
           String newName = "I" + namer.getNextName() + "." + extension;
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java
index 2a9ea55..2cdaa64 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java
@@ -20,12 +20,14 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.Constants.IMPORT_MAPPINGS_FILE;
+import static org.apache.accumulo.manager.tableOps.tableExport.ExportTable.VERSION_2;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedReader;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.zip.ZipEntry;
@@ -41,7 +43,8 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -94,7 +97,9 @@
 
     VolumeManager fs = manager.getVolumeManager();
 
-    try (BatchWriter mbw = manager.getContext().createBatchWriter(MetadataTable.NAME);
+    try (
+        BatchWriter mbw =
+            manager.getContext().createBatchWriter(AccumuloTable.METADATA.tableName());
         ZipInputStream zis = new ZipInputStream(fs.open(path))) {
 
       Map<String,String> fileNameMappings = new HashMap<>();
@@ -127,7 +132,17 @@
             Text cq;
 
             if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-              String oldName = new Path(key.getColumnQualifier().toString()).getName();
+              StoredTabletFile exportedRef;
+              var dataFileCQ = key.getColumnQualifier().toString();
+              if (tableInfo.exportedVersion == null || tableInfo.exportedVersion < VERSION_2) {
+                // written without fenced range information (accumulo < 3.1), use default
+                // (null,null)
+                exportedRef = StoredTabletFile.of(new Path(dataFileCQ));
+              } else {
+                exportedRef = StoredTabletFile.of(key.getColumnQualifier());
+              }
+
+              String oldName = exportedRef.getFileName();
               String newName = fileNameMappings.get(oldName);
 
               if (newName == null) {
@@ -136,7 +151,9 @@
                     "File " + oldName + " does not exist in import dir");
               }
 
-              cq = new Text(newName);
+              // Copy over the range for the new file
+              cq = StoredTabletFile.of(URI.create(newName), exportedRef.getRange())
+                  .getMetadataText();
             } else {
               cq = key.getColumnQualifier();
             }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java
index 7c50d82..eb7ab83 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java
@@ -22,12 +22,11 @@
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
@@ -35,6 +34,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class ShutdownTServer extends ManagerRepo {
 
   private static final long serialVersionUID = 2L;
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java
new file mode 100644
index 0000000..2449355
--- /dev/null
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/PreUpgradeValidation.java
@@ -0,0 +1,120 @@
+/*
+ * 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.manager.upgrade;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.manager.EventCoordinator;
+import org.apache.accumulo.server.AccumuloDataVersion;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+/**
+ * Provide checks before upgraders run that can perform checks that the environment from previous
+ * versions match expectations. Checks include:
+ * <ul>
+ * <li>ACL validation of ZooKeeper nodes</li>
+ * </ul>
+ */
+public class PreUpgradeValidation {
+
+  private final static Logger log = LoggerFactory.getLogger(PreUpgradeValidation.class);
+
+  public void validate(final ServerContext context, final EventCoordinator eventCoordinator) {
+    int cv = AccumuloDataVersion.getCurrentVersion(context);
+    if (cv == AccumuloDataVersion.get()) {
+      log.debug("already at current data version: {}, skipping validation", cv);
+      return;
+    }
+    validateACLs(context);
+  }
+
+  private void validateACLs(ServerContext context) {
+
+    final AtomicBoolean aclErrorOccurred = new AtomicBoolean(false);
+    final ZooReaderWriter zrw = context.getZooReaderWriter();
+    final ZooKeeper zk = zrw.getZooKeeper();
+    final String rootPath = context.getZooKeeperRoot();
+    final Set<String> users = Set.of("accumulo", "anyone");
+
+    log.info("Starting validation on ZooKeeper ACLs");
+
+    try {
+      ZKUtil.visitSubTreeDFS(zk, rootPath, false, (rc, path, ctx, name) -> {
+        try {
+          final List<ACL> acls = zk.getACL(path, new Stat());
+          if (!hasAllPermissions(users, acls)) {
+            log.error(
+                "ZNode at {} does not have an ACL that allows accumulo to write to it. ZNode ACL will need to be modified. Current ACLs: {}",
+                path, acls);
+            aclErrorOccurred.set(true);
+          }
+        } catch (KeeperException | InterruptedException e) {
+          log.error("Error getting ACL for path: {}", path, e);
+          aclErrorOccurred.set(true);
+        }
+      });
+      if (aclErrorOccurred.get()) {
+        throw new RuntimeException(
+            "Upgrade precondition failed! ACLs will need to be modified for some ZooKeeper nodes. "
+                + "Check the log for specific failed paths, check ZooKeeper troubleshooting in user documentation "
+                + "for instructions on how to fix.");
+      }
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException("Upgrade Failed! Error validating nodes under " + rootPath, e);
+    }
+    log.info("Successfully completed validation on ZooKeeper ACLs");
+  }
+
+  private static boolean hasAllPermissions(final Set<String> users, final List<ACL> acls) {
+    return acls.stream()
+        .anyMatch(a -> users.contains(extractAuthName(a)) && a.getPerms() == ZooDefs.Perms.ALL);
+  }
+
+  private static String extractAuthName(ACL acl) {
+    Objects.requireNonNull(acl, "provided ACL cannot be null");
+    try {
+      return acl.getId().getId().trim().split(":")[0];
+    } catch (Exception ex) {
+      log.debug("Invalid ACL passed, cannot parse id from '{}'", acl);
+      return "";
+    }
+  }
+
+  @SuppressFBWarnings(value = "DM_EXIT",
+      justification = "Want to immediately stop all threads on upgrade error")
+  private void fail(Exception e) {
+    log.error("FATAL: Error performing pre-upgrade checks", e);
+    System.exit(1);
+  }
+
+}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/RenameMasterDirInZK.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/RenameMasterDirInZK.java
deleted file mode 100644
index 3db373f..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/RenameMasterDirInZK.java
+++ /dev/null
@@ -1,71 +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.manager.upgrade;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A utility to handle the renaming of "/masters" to "/managers" in Zookeeper when upgrading from a
- * 2.0 (or earlier) to 2.1 instance. This utility is invoked automatically by
- * {@link org.apache.accumulo.manager.state.SetGoalState} (which normally runs first as a part of
- * accumulo startup scripts). However, if a user is not using the standard scripts or wishes to
- * perform the upgrade as a separate process, this utility can be invoked with:
- *
- * <pre>
- * {@code
- * bin/accumulo org.apache.accumulo.manager.upgrade.RenameMasterDirInZK
- * }
- * </pre>
- */
-public class RenameMasterDirInZK {
-  private static final Logger LOG = LoggerFactory.getLogger(RenameMasterDirInZK.class);
-
-  public static void main(String[] args) {
-    var ctx = new ServerContext(SiteConfiguration.auto());
-    if (!renameMasterDirInZK(ctx)) {
-      LOG.info(
-          "Masters directory in ZooKeeper has already been renamed to managers. No action was taken.");
-    }
-  }
-
-  public static boolean renameMasterDirInZK(ServerContext context) {
-    final ZooReaderWriter zoo = context.getZooReaderWriter();
-    final String mastersZooDir = context.getZooKeeperRoot() + "/masters";
-    final String managersZooDir = context.getZooKeeperRoot() + Constants.ZMANAGERS;
-    try {
-      boolean mastersDirExists = zoo.exists(mastersZooDir);
-      if (mastersDirExists) {
-        LOG.info("Copying ZooKeeper directory {} to {}.", mastersZooDir, managersZooDir);
-        zoo.recursiveCopyPersistentOverwrite(mastersZooDir, managersZooDir);
-        LOG.info("Deleting ZooKeeper directory {}.", mastersZooDir);
-        zoo.recursiveDelete(mastersZooDir, ZooUtil.NodeMissingPolicy.SKIP);
-      }
-      return mastersDirExists;
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException("Unable to rename " + mastersZooDir + " in ZooKeeper", e);
-    }
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
index 11cb713..84e69d5 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
@@ -19,10 +19,13 @@
 package org.apache.accumulo.manager.upgrade;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.server.AccumuloDataVersion.REMOVE_DEPRECATIONS_FOR_VERSION_3;
+import static org.apache.accumulo.server.AccumuloDataVersion.ROOT_TABLET_META_CHANGES;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Objects;
 import java.util.TreeMap;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Future;
@@ -56,7 +59,7 @@
 
   public enum UpgradeStatus {
     /**
-     * This signifies the upgrade status is in the process of being determined. Its best to assume
+     * This signifies the upgrade status is in the process of being determined. It is best to assume
      * nothing is upgraded when seeing this.
      */
     INITIAL {
@@ -118,15 +121,14 @@
     public abstract boolean isParentLevelUpgraded(KeyExtent extent);
   }
 
-  private static Logger log = LoggerFactory.getLogger(UpgradeCoordinator.class);
+  private static final Logger log = LoggerFactory.getLogger(UpgradeCoordinator.class);
 
   private int currentVersion;
-
-  // unmodifiable map of "current version" -> upgrader to next version.
+  // map of "current version" -> upgrader to next version.
   // Sorted so upgrades execute in order from the oldest supported data version to current
-  private Map<Integer,Upgrader> upgraders =
-      Collections.unmodifiableMap(new TreeMap<>(Map.of(AccumuloDataVersion.SHORTEN_RFILE_KEYS,
-          new Upgrader8to9(), AccumuloDataVersion.CRYPTO_CHANGES, new Upgrader9to10())));
+  private final Map<Integer,Upgrader> upgraders =
+      Collections.unmodifiableMap(new TreeMap<>(Map.of(ROOT_TABLET_META_CHANGES,
+          new Upgrader10to11(), REMOVE_DEPRECATIONS_FOR_VERSION_3, new Upgrader11to12())));
 
   private volatile UpgradeStatus status;
 
@@ -158,9 +160,7 @@
         "Not currently in a suitable state to do zookeeper upgrade %s", status);
 
     try {
-      int cv = context.getServerDirs()
-          .getAccumuloPersistentVersion(context.getVolumeManager().getFirst());
-      ServerContext.ensureDataVersionCompatible(cv);
+      int cv = AccumuloDataVersion.getCurrentVersion(context);
       this.currentVersion = cv;
 
       if (cv == AccumuloDataVersion.get()) {
@@ -172,8 +172,12 @@
         abortIfFateTransactions(context);
 
         for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-          log.info("Upgrading Zookeeper from data version {}", v);
-          upgraders.get(v).upgradeZookeeper(context);
+          log.info("Upgrading Zookeeper - current version {} as step towards target version {}", v,
+              AccumuloDataVersion.get());
+          var upgrader = upgraders.get(v);
+          Objects.requireNonNull(upgrader,
+              "upgrade ZooKeeper: failed to find upgrader for version " + currentVersion);
+          upgrader.upgradeZookeeper(context);
         }
       }
 
@@ -199,13 +203,22 @@
           .withQueue(new SynchronousQueue<>()).build().submit(() -> {
             try {
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Root from data version {}", v);
+                log.info("Upgrading Root - current version {} as step towards target version {}", v,
+                    AccumuloDataVersion.get());
+                var upgrader = upgraders.get(v);
+                Objects.requireNonNull(upgrader,
+                    "upgrade root: failed to find root upgrader for version " + currentVersion);
                 upgraders.get(v).upgradeRoot(context);
               }
               setStatus(UpgradeStatus.UPGRADED_ROOT, eventCoordinator);
 
               for (int v = currentVersion; v < AccumuloDataVersion.get(); v++) {
-                log.info("Upgrading Metadata from data version {}", v);
+                log.info(
+                    "Upgrading Metadata - current version {} as step towards target version {}", v,
+                    AccumuloDataVersion.get());
+                var upgrader = upgraders.get(v);
+                Objects.requireNonNull(upgrader,
+                    "upgrade metadata: failed to find upgrader for version " + currentVersion);
                 upgraders.get(v).upgradeMetadata(context);
               }
               setStatus(UpgradeStatus.UPGRADED_METADATA, eventCoordinator);
@@ -279,13 +292,13 @@
    * need to make sure there are no queued transactions from a previous version before continuing an
    * upgrade. The status of the operations is irrelevant; those in SUCCESSFUL status cause the same
    * problem as those just queued.
-   *
+   * <p>
    * Note that the Manager should not allow write access to Fate until after all upgrade steps are
    * complete.
-   *
+   * <p>
    * Should be called as a guard before performing any upgrade steps, after determining that an
    * upgrade is needed.
-   *
+   * <p>
    * see ACCUMULO-2519
    */
   @SuppressFBWarnings(value = "DM_EXIT",
@@ -298,7 +311,7 @@
         throw new AccumuloException("Aborting upgrade because there are"
             + " outstanding FATE transactions from a previous Accumulo version."
             + " You can start the tservers and then use the shell to delete completed "
-            + " transactions. If there are uncomplete transactions, you will need to roll"
+            + " transactions. If there are incomplete transactions, you will need to roll"
             + " back and fix those issues. Please see the following page for more information: "
             + " https://accumulo.apache.org/docs/2.x/troubleshooting/advanced#upgrade-issues");
       }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java
new file mode 100644
index 0000000..4629f48
--- /dev/null
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader10to11.java
@@ -0,0 +1,269 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
+import static org.apache.accumulo.server.util.MetadataTableUtil.EMPTY_TEXT;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.InstanceId;
+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.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader10to11 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11.class);
+
+  // Included for upgrade code usage any other usage post 3.0 should not be used.
+  private static final TableId REPLICATION_ID = TableId.of("+rep");
+
+  private static final Range REP_TABLE_RANGE =
+      new Range(REPLICATION_ID.canonical() + ";", true, REPLICATION_ID.canonical() + "<", true);
+
+  // copied from MetadataSchema 2.1 (removed in 3.0)
+  private static final Range REP_WAL_RANGE =
+      new Range(RESERVED_PREFIX + "repl", true, RESERVED_PREFIX + "repm", false);
+
+  public Upgrader10to11() {
+    super();
+  }
+
+  @Override
+  public void upgradeZookeeper(final ServerContext context) {
+    log.info("upgrade of ZooKeeper entries");
+
+    var zrw = context.getZooReaderWriter();
+    var iid = context.getInstanceID();
+
+    // if the replication base path (../tables/+rep) assume removed or never existed.
+    if (!checkReplicationTableInZk(iid, zrw)) {
+      log.debug("replication table root node does not exist in ZooKeeper - nothing to do");
+      return;
+    }
+
+    // if the replication table is online - stop. There could be data in transit.
+    if (!checkReplicationOffline(iid, zrw)) {
+      throw new IllegalStateException(
+          "Replication table is not offline. Cannot continue with upgrade that will remove replication with replication active");
+    }
+
+    cleanMetaConfig(iid, context.getPropStore());
+
+    deleteReplicationTableZkEntries(zrw, iid);
+
+  }
+
+  @Override
+  public void upgradeRoot(final ServerContext context) {
+    log.info("upgrade root - skipping, nothing to do");
+  }
+
+  @Override
+  public void upgradeMetadata(final ServerContext context) {
+    log.info("upgrade metadata entries");
+    List<String> replTableFiles = readReplFilesFromMetadata(context);
+    deleteReplMetadataEntries(context);
+    deleteReplTableFiles(context, replTableFiles);
+  }
+
+  List<String> readReplFilesFromMetadata(final ServerContext context) {
+    List<String> results = new ArrayList<>();
+    try (Scanner scanner =
+        context.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+      scanner.setRange(REP_TABLE_RANGE);
+      for (Map.Entry<Key,Value> entry : scanner) {
+        String f = entry.getKey()
+            .getColumnQualifier(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME).toString();
+        results.add(f);
+      }
+    } catch (TableNotFoundException ex) {
+      throw new IllegalStateException("failed to read replication files from metadata", ex);
+    }
+    return results;
+  }
+
+  void deleteReplTableFiles(final ServerContext context, final List<String> replTableFiles) {
+    // short circuit if there are no files
+    if (replTableFiles.isEmpty()) {
+      return;
+    }
+    // write delete mutations
+    boolean haveFailures = false;
+    try (BatchWriter writer = context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
+      for (String filename : replTableFiles) {
+        Mutation m = createDelMutation(filename);
+        log.debug("Adding delete marker for file: {}", filename);
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException ex) {
+      log.debug("Failed to write delete marker {}", ex.getMessage());
+      haveFailures = true;
+    } catch (TableNotFoundException ex) {
+      throw new IllegalStateException("failed to read replication files from metadata", ex);
+    }
+    if (haveFailures) {
+      throw new IllegalStateException(
+          "deletes rejected adding deletion marker for replication file entries, check log");
+    }
+  }
+
+  private Mutation createDelMutation(String path) {
+    Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.encodeRow(path)));
+    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, MetadataSchema.DeletesSection.SkewedKeyValue.NAME);
+    return delFlag;
+  }
+
+  /**
+   * remove +rep entries from metadata.
+   */
+  private void deleteReplMetadataEntries(final ServerContext context) {
+    try (BatchDeleter deleter =
+        context.createBatchDeleter(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY, 10)) {
+      deleter.setRanges(List.of(REP_TABLE_RANGE, REP_WAL_RANGE));
+      deleter.delete();
+    } catch (TableNotFoundException | MutationsRejectedException ex) {
+      throw new IllegalStateException("failed to remove replication info from metadata table", ex);
+    }
+  }
+
+  private boolean checkReplicationTableInZk(final InstanceId iid, final ZooReaderWriter zrw) {
+    try {
+      String path = buildRepTablePath(iid);
+      return zrw.exists(path);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("ZooKeeper error - cannot determine replication table status",
+          ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted reading replication state from ZooKeeper", ex);
+    }
+  }
+
+  /**
+   * To protect against removing replication information if replication is being used and possible
+   * active, check the replication table state in Zookeeper to see if it is ONLINE (active) or
+   * OFFLINE (inactive). If the state node does not exist, then the status is considered as OFFLINE.
+   *
+   * @return true if the replication table state is OFFLINE, false otherwise
+   */
+  private boolean checkReplicationOffline(final InstanceId iid, final ZooReaderWriter zrw) {
+    try {
+      String path = buildRepTablePath(iid) + ZTABLE_STATE;
+      byte[] bytes = zrw.getData(path);
+      if (bytes != null && bytes.length > 0) {
+        String status = new String(bytes, UTF_8);
+        return TableState.OFFLINE.name().equals(status);
+      }
+      return false;
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("ZooKeeper error - cannot determine replication table status",
+          ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted reading replication state from ZooKeeper", ex);
+    }
+  }
+
+  /**
+   * Utility method to build the ZooKeeper replication table path. The path resolves to
+   * {@code /accumulo/INSTANCE_ID/tables/+rep}
+   */
+  static String buildRepTablePath(final InstanceId iid) {
+    return ZooUtil.getRoot(iid) + ZTABLES + "/" + REPLICATION_ID.canonical();
+  }
+
+  private void deleteReplicationTableZkEntries(ZooReaderWriter zrw, InstanceId iid) {
+    String repTablePath = buildRepTablePath(iid);
+    try {
+      zrw.recursiveDelete(repTablePath, ZooUtil.NodeMissingPolicy.SKIP);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "ZooKeeper error - failed recursive deletion on " + repTablePath, ex);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted deleting " + repTablePath + " from ZooKeeper",
+          ex);
+    }
+  }
+
+  private void cleanMetaConfig(final InstanceId iid, final PropStore propStore) {
+    PropStoreKey<TableId> metaKey = TablePropKey.of(iid, AccumuloTable.METADATA.tableId());
+    var p = propStore.get(metaKey);
+    var props = p.asMap();
+    List<String> filtered = filterReplConfigKeys(props.keySet());
+    // add replication status formatter to remove list.
+    String v = props.get("table.formatter");
+    if (v != null && v.compareTo("org.apache.accumulo.server.replication.StatusFormatter") == 0) {
+      filtered.add("table.formatter");
+    }
+
+    if (filtered.size() > 0) {
+      log.trace("Upgrade filtering replication iterators for id: {}", metaKey);
+      propStore.removeProperties(metaKey, filtered);
+    }
+  }
+
+  /**
+   * Return a list of property keys that match replication iterator settings. This is specifically a
+   * narrow filter to avoid potential matches with user define or properties that contain
+   * replication in the property name (specifically table.file.replication which set hdfs block
+   * replication.)
+   */
+  private List<String> filterReplConfigKeys(Set<String> keys) {
+    String REPL_ITERATOR_PATTERN = "^table\\.iterator\\.(majc|minc|scan)\\.replcombiner$";
+    String REPL_COLUMN_PATTERN =
+        "^table\\.iterator\\.(majc|minc|scan)\\.replcombiner\\.opt\\.columns$";
+
+    Pattern p = Pattern.compile("(" + REPL_ITERATOR_PATTERN + "|" + REPL_COLUMN_PATTERN + ")");
+
+    return keys.stream().filter(e -> p.matcher(e).find()).collect(Collectors.toList());
+  }
+}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
new file mode 100644
index 0000000..0c5e086
--- /dev/null
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
@@ -0,0 +1,212 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+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.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+public class Upgrader11to12 implements Upgrader {
+
+  private static final Logger log = LoggerFactory.getLogger(Upgrader11to12.class);
+
+  @SuppressWarnings("deprecation")
+  private static final Text CHOPPED = ChoppedColumnFamily.NAME;
+
+  @VisibleForTesting
+  static final Set<Text> UPGRADE_FAMILIES =
+      Set.of(DataFileColumnFamily.NAME, CHOPPED, ExternalCompactionColumnFamily.NAME);
+
+  @Override
+  public void upgradeZookeeper(@NonNull ServerContext context) {
+    log.debug("Upgrade ZooKeeper: upgrading to data version {}", METADATA_FILE_JSON_ENCODING);
+    var rootBase = ZooUtil.getRoot(context.getInstanceID()) + ZROOT_TABLET;
+
+    try {
+      var zrw = context.getZooReaderWriter();
+      Stat stat = new Stat();
+      byte[] rootData = zrw.getData(rootBase, stat);
+
+      String json = new String(rootData, UTF_8);
+
+      var rtm = new RootTabletMetadata(json);
+
+      TreeMap<Key,Value> entries = new TreeMap<>();
+      rtm.toKeyValues().filter(e -> UPGRADE_FAMILIES.contains(e.getKey().getColumnFamily()))
+          .forEach(entry -> entries.put(entry.getKey(), entry.getValue()));
+      ArrayList<Mutation> mutations = new ArrayList<>();
+
+      processReferences(mutations::add, entries.entrySet(), "root_table_metadata");
+
+      Preconditions.checkState(mutations.size() <= 1);
+
+      if (!mutations.isEmpty()) {
+        log.info("Root metadata in ZooKeeper before upgrade: {}", json);
+        rtm.update(mutations.get(0));
+        zrw.overwritePersistentData(rootBase, rtm.toJson().getBytes(UTF_8), stat.getVersion());
+        log.info("Root metadata in ZooKeeper after upgrade: {}", rtm.toJson());
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(
+          "Could not read root metadata from ZooKeeper due to interrupt", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException(
+          "Could not read or write root metadata in ZooKeeper because of ZooKeeper exception", ex);
+    }
+  }
+
+  interface MutationWriter {
+    void addMutation(Mutation m) throws MutationsRejectedException;
+  }
+
+  @Override
+  public void upgradeRoot(@NonNull ServerContext context) {
+    log.debug("Upgrade root: upgrading to data version {}", METADATA_FILE_JSON_ENCODING);
+    var rootName = Ample.DataLevel.METADATA.metaTable();
+    upgradeTabletsMetadata(context, rootName);
+  }
+
+  @Override
+  public void upgradeMetadata(@NonNull ServerContext context) {
+    log.debug("Upgrade metadata: upgrading to data version {}", METADATA_FILE_JSON_ENCODING);
+    var metaName = Ample.DataLevel.USER.metaTable();
+    upgradeTabletsMetadata(context, metaName);
+  }
+
+  private void upgradeTabletsMetadata(@NonNull ServerContext context, String metaName) {
+    // not using ample to avoid StoredTabletFile because old file ref is incompatible
+    try (BatchWriter batchWriter = context.createBatchWriter(metaName); Scanner scanner =
+        new IsolatedScanner(context.createScanner(metaName, Authorizations.EMPTY))) {
+      UPGRADE_FAMILIES.forEach(scanner::fetchColumnFamily);
+      scanner.setRange(MetadataSchema.TabletsSection.getRange());
+      processReferences(batchWriter::addMutation, scanner, metaName);
+    } catch (TableNotFoundException ex) {
+      throw new IllegalStateException("Failed to find table " + metaName, ex);
+    } catch (MutationsRejectedException mex) {
+      log.warn("Failed to update reference for table: " + metaName);
+      log.warn("Constraint violations: {}", mex.getConstraintViolationSummaries());
+      throw new IllegalStateException("Failed to process table: " + metaName, mex);
+    }
+  }
+
+  void processReferences(MutationWriter batchWriter, Iterable<Map.Entry<Key,Value>> scanner,
+      String tableName) {
+    try {
+      Mutation update = null;
+      for (Map.Entry<Key,Value> entry : scanner) {
+        Key key = entry.getKey();
+        Value value = entry.getValue();
+        Preconditions.checkState(key.getColumnVisibilityData().length() == 0,
+            "Expected empty visibility, saw %s ", key.getColumnVisibilityData());
+        // on new row, write current mutation and prepare a new one.
+        Text r = key.getRow();
+        if (update == null) {
+          update = new Mutation(r);
+        } else if (!Arrays.equals(update.getRow(), TextUtil.getBytes(r))) {
+          if (log.isTraceEnabled()) {
+            log.trace("table: {}, update: {}", tableName, update.prettyPrint());
+          }
+          if (!update.getUpdates().isEmpty()) {
+            batchWriter.addMutation(update);
+          }
+          update = new Mutation(r);
+        }
+
+        var family = key.getColumnFamily();
+        if (family.equals(DataFileColumnFamily.NAME)) {
+          upgradeDataFileCF(key, value, update);
+        } else if (family.equals(CHOPPED)) {
+          log.warn(
+              "Deleting chopped reference from:{}. Previous split or delete may not have completed cleanly. Ref: {}",
+              tableName, key.getRow());
+          update.at().family(CHOPPED).qualifier(CHOPPED).delete();
+        } else if (family.equals(ExternalCompactionColumnFamily.NAME)) {
+          log.debug(
+              "Deleting external compaction reference from:{}. Previous compaction may not have completed. Ref: {}",
+              tableName, key.getRow());
+          update.at().family(ExternalCompactionColumnFamily.NAME)
+              .qualifier(key.getColumnQualifier()).delete();
+        } else {
+          throw new IllegalStateException("Processing: " + tableName
+              + " Received unexpected column family processing references: " + family);
+        }
+      }
+      // send last mutation
+      if (update != null && !update.getUpdates().isEmpty()) {
+        log.trace("table: {}, update: {}", tableName, update.prettyPrint());
+        batchWriter.addMutation(update);
+      }
+    } catch (MutationsRejectedException mex) {
+      log.warn("Failed to update reference for table: " + tableName);
+      log.warn("Constraint violations: {}", mex.getConstraintViolationSummaries());
+      throw new IllegalStateException("Failed to process table: " + tableName, mex);
+    }
+  }
+
+  @VisibleForTesting
+  static void upgradeDataFileCF(final Key key, final Value value, final Mutation m) {
+    String file = key.getColumnQualifier().toString();
+    // filter out references if they are in the correct format already.
+    boolean needsConversion = StoredTabletFile.fileNeedsConversion(file);
+    log.trace("file: {} needs conversion: {}", file, needsConversion);
+    if (needsConversion) {
+      var fileJson = StoredTabletFile.of(new Path(file)).getMetadataText();
+      m.at().family(DataFileColumnFamily.STR_NAME).qualifier(fileJson).put(value);
+      m.at().family(DataFileColumnFamily.STR_NAME).qualifier(file).delete();
+    }
+  }
+
+}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader8to9.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader8to9.java
deleted file mode 100644
index 2250a51..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader8to9.java
+++ /dev/null
@@ -1,44 +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.manager.upgrade;
-
-import org.apache.accumulo.server.AccumuloDataVersion;
-import org.apache.accumulo.server.ServerContext;
-
-/**
- * See {@link AccumuloDataVersion#CRYPTO_CHANGES}
- */
-public class Upgrader8to9 implements Upgrader {
-
-  @Override
-  public void upgradeZookeeper(ServerContext context) {
-    // There is no action that needs to be taken for zookeeper
-  }
-
-  @Override
-  public void upgradeRoot(ServerContext context) {
-    // There is no action that needs to be taken for metadata
-  }
-
-  @Override
-  public void upgradeMetadata(ServerContext context) {
-    // There is no action that needs to be taken for metadata
-  }
-
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
deleted file mode 100644
index 6bf9e32..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
+++ /dev/null
@@ -1,872 +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.manager.upgrade;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
-import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET_GC_CANDIDATES;
-import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
-import static org.apache.accumulo.server.util.MetadataTableUtil.EMPTY_TEXT;
-
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.BiConsumer;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.conf.Property;
-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.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-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.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-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.TServerInstance;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection.SkewedKeyValue;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataTime;
-import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.TablePropKey;
-import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.gc.AllVolumesDirectory;
-import org.apache.accumulo.server.gc.GcVolumeUtil;
-import org.apache.accumulo.server.metadata.RootGcCandidates;
-import org.apache.accumulo.server.metadata.TabletMutatorBase;
-import org.apache.accumulo.server.util.PropUtil;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-/**
- * Handles upgrading from 2.0 to 2.1.
- * <ul>
- * <li><strong>Rename master properties (Issue
- * <a href="https://github.com/apache/accumulo/issues/1640">#1640</a>):</strong> Rename any
- * ZooKeeper system properties that start with "master." to the equivalent property starting with
- * "manager." instead (see the {@code renameOldMasterPropsinZK(ServerContext)} method). Note that
- * this change was part of a larger effort to replace references to master with manager. See issues
- * <a href="https://github.com/apache/accumulo/issues/1641">#1641</a>,
- * <a href="https://github.com/apache/accumulo/issues/1642">#1642</a>, and
- * <a href="https://github.com/apache/accumulo/issues/1643">#1643</a> as well.</li>
- * </ul>
- */
-public class Upgrader9to10 implements Upgrader {
-
-  private static final Logger log = LoggerFactory.getLogger(Upgrader9to10.class);
-
-  public static final String ZROOT_TABLET_LOCATION = ZROOT_TABLET + "/location";
-  public static final String ZROOT_TABLET_FUTURE_LOCATION = ZROOT_TABLET + "/future_location";
-  public static final String ZROOT_TABLET_LAST_LOCATION = ZROOT_TABLET + "/lastlocation";
-  public static final String ZROOT_TABLET_WALOGS = ZROOT_TABLET + "/walogs";
-  public static final String ZROOT_TABLET_CURRENT_LOGS = ZROOT_TABLET + "/current_logs";
-  public static final String ZROOT_TABLET_PATH = ZROOT_TABLET + "/dir";
-  public static final Value UPGRADED = SkewedKeyValue.NAME;
-  public static final String OLD_DELETE_PREFIX = "~del";
-
-  // effectively an 8MB batch size, since this number is the number of Chars
-  public static final long CANDIDATE_BATCH_SIZE = 4_000_000;
-
-  @Override
-  public void upgradeZookeeper(ServerContext context) {
-    validateACLs(context);
-    upgradePropertyStorage(context);
-    setMetaTableProps(context);
-    upgradeRootTabletMetadata(context);
-    createExternalCompactionNodes(context);
-    // special case where old files need to be deleted
-    dropSortedMapWALFiles(context);
-    createScanServerNodes(context);
-  }
-
-  private static String extractAuthName(ACL acl) {
-    Objects.requireNonNull(acl, "provided ACL cannot be null");
-    try {
-      return acl.getId().getId().trim().split(":")[0];
-    } catch (Exception ex) {
-      log.debug("Invalid ACL passed, cannot parse id from '{}'", acl);
-      return "";
-    }
-  }
-
-  private static boolean hasAllPermissions(final Set<String> users, final List<ACL> acls) {
-    return acls.stream()
-        .anyMatch(a -> users.contains(extractAuthName(a)) && a.getPerms() == ZooDefs.Perms.ALL);
-  }
-
-  private void validateACLs(ServerContext context) {
-
-    final AtomicBoolean aclErrorOccurred = new AtomicBoolean(false);
-    final ZooReaderWriter zrw = context.getZooReaderWriter();
-    final ZooKeeper zk = zrw.getZooKeeper();
-    final String rootPath = context.getZooKeeperRoot();
-    final Set<String> users = Set.of("accumulo", "anyone");
-
-    try {
-      ZKUtil.visitSubTreeDFS(zk, rootPath, false, (rc, path, ctx, name) -> {
-        try {
-          final List<ACL> acls = zk.getACL(path, new Stat());
-          if (!hasAllPermissions(users, acls)) {
-            log.error(
-                "ZNode at {} does not have an ACL that allows accumulo to write to it. ZNode ACL will need to be modified. Current ACLs: {}",
-                path, acls);
-            aclErrorOccurred.set(true);
-          }
-        } catch (KeeperException | InterruptedException e) {
-          log.error("Error getting ACL for path: {}", path, e);
-          aclErrorOccurred.set(true);
-        }
-      });
-      if (aclErrorOccurred.get()) {
-        throw new RuntimeException(
-            "Upgrade precondition failed! ACLs will need to be modified for some ZooKeeper nodes. "
-                + "Check the log for specific failed paths, check ZooKeeper troubleshooting in user documentation "
-                + "for instructions on how to fix.");
-      }
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException("Upgrade Failed! Error validating nodes under " + rootPath, e);
-    }
-  }
-
-  @Override
-  public void upgradeRoot(ServerContext context) {
-    upgradeRelativePaths(context, Ample.DataLevel.METADATA);
-    upgradeDirColumns(context, Ample.DataLevel.METADATA);
-    upgradeFileDeletes(context, Ample.DataLevel.METADATA);
-  }
-
-  @Override
-  public void upgradeMetadata(ServerContext context) {
-    upgradeRelativePaths(context, Ample.DataLevel.USER);
-    upgradeDirColumns(context, Ample.DataLevel.USER);
-    upgradeFileDeletes(context, Ample.DataLevel.USER);
-  }
-
-  /**
-   * Convert system properties (if necessary) and all table properties to a single node
-   */
-  private void upgradePropertyStorage(ServerContext context) {
-    log.info("Starting property conversion");
-    ConfigPropertyUpgrader configUpgrader = new ConfigPropertyUpgrader();
-    configUpgrader.doUpgrade(context.getInstanceID(), context.getZooReaderWriter());
-    log.info("Completed property conversion");
-  }
-
-  /**
-   * Setup properties for External compactions.
-   */
-  private void setMetaTableProps(ServerContext context) {
-    try {
-      // sets the compaction dispatcher props for the given table and service name
-      BiConsumer<TableId,String> setDispatcherProps =
-          (TableId tableId, String dispatcherService) -> {
-            var dispatcherPropsMap = Map.of(Property.TABLE_COMPACTION_DISPATCHER.getKey(),
-                SimpleCompactionDispatcher.class.getName(),
-                Property.TABLE_COMPACTION_DISPATCHER_OPTS.getKey() + "service", dispatcherService);
-            PropUtil.setProperties(context, TablePropKey.of(context, tableId), dispatcherPropsMap);
-          };
-
-      // root compaction props
-      setDispatcherProps.accept(RootTable.ID, "root");
-      // metadata compaction props
-      setDispatcherProps.accept(MetadataTable.ID, "meta");
-    } catch (IllegalStateException ex) {
-      throw new RuntimeException("Unable to set system table properties", ex);
-    }
-  }
-
-  private void createScanServerNodes(ServerContext context) {
-    final byte[] EMPTY_BYTE_ARRAY = new byte[0];
-    try {
-      context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + Constants.ZSSERVERS, EMPTY_BYTE_ARRAY,
-          NodeExistsPolicy.SKIP);
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException("Unable to create scan server paths", e);
-    }
-  }
-
-  private void createExternalCompactionNodes(ServerContext context) {
-
-    final byte[] EMPTY_BYTE_ARRAY = new byte[0];
-    try {
-      context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + Constants.ZCOORDINATOR, EMPTY_BYTE_ARRAY,
-          NodeExistsPolicy.SKIP);
-      context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK, EMPTY_BYTE_ARRAY,
-          NodeExistsPolicy.SKIP);
-      context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + Constants.ZCOMPACTORS, EMPTY_BYTE_ARRAY,
-          NodeExistsPolicy.SKIP);
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException("Unable to create external compaction paths", e);
-    }
-  }
-
-  /**
-   * Improvements to the metadata and root tables were made in this version. See pull request
-   * <a href="https://github.com/apache/accumulo/pull/1174">#1174</a> for more details.
-   */
-  private void upgradeRootTabletMetadata(ServerContext context) {
-    String rootMetaSer = getFromZK(context, ZROOT_TABLET);
-
-    if (rootMetaSer == null || rootMetaSer.isEmpty()) {
-      String dir = getFromZK(context, ZROOT_TABLET_PATH);
-      List<LogEntry> logs = getRootLogEntries(context);
-
-      TServerInstance last = getLocation(context, ZROOT_TABLET_LAST_LOCATION);
-      TServerInstance future = getLocation(context, ZROOT_TABLET_FUTURE_LOCATION);
-      TServerInstance current = getLocation(context, ZROOT_TABLET_LOCATION);
-
-      UpgradeMutator tabletMutator = new UpgradeMutator(context);
-
-      tabletMutator.putPrevEndRow(RootTable.EXTENT.prevEndRow());
-
-      tabletMutator.putDirName(upgradeDirColumn(dir));
-
-      if (last != null) {
-        tabletMutator.putLocation(Location.last(last));
-      }
-
-      if (future != null) {
-        tabletMutator.putLocation(Location.future(future));
-      }
-
-      if (current != null) {
-        tabletMutator.putLocation(Location.current(current));
-      }
-
-      logs.forEach(tabletMutator::putWal);
-
-      Map<String,DataFileValue> files = cleanupRootTabletFiles(context.getVolumeManager(), dir);
-      files.forEach((path, dfv) -> tabletMutator.putFile(new TabletFile(new Path(path)), dfv));
-
-      tabletMutator.putTime(computeRootTabletTime(context, files.keySet()));
-
-      tabletMutator.mutate();
-    }
-
-    try {
-      context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + ZROOT_TABLET_GC_CANDIDATES,
-          new RootGcCandidates().toJson().getBytes(UTF_8), NodeExistsPolicy.SKIP);
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException(e);
-    }
-
-    // this operation must be idempotent, so deleting after updating is very important
-
-    delete(context, ZROOT_TABLET_CURRENT_LOGS);
-    delete(context, ZROOT_TABLET_FUTURE_LOCATION);
-    delete(context, ZROOT_TABLET_LAST_LOCATION);
-    delete(context, ZROOT_TABLET_LOCATION);
-    delete(context, ZROOT_TABLET_WALOGS);
-    delete(context, ZROOT_TABLET_PATH);
-  }
-
-  private static class UpgradeMutator extends TabletMutatorBase {
-
-    private final ServerContext context;
-
-    UpgradeMutator(ServerContext context) {
-      super(context, RootTable.EXTENT);
-      this.context = context;
-    }
-
-    @Override
-    public void mutate() {
-      Mutation mutation = getMutation();
-
-      try {
-        context.getZooReaderWriter().mutateOrCreate(
-            context.getZooKeeperRoot() + RootTable.ZROOT_TABLET, new byte[0], currVal -> {
-              // Earlier, it was checked that root tablet metadata did not exists. However the
-              // earlier check does handle race conditions. Race conditions are unexpected. This is
-              // a sanity check when making the update in ZK using compare and set. If this fails
-              // and its not a bug, then its likely some concurrency issue. For example two managers
-              // concurrently running upgrade could cause this to fail.
-              Preconditions.checkState(currVal.length == 0,
-                  "Expected root tablet metadata to be empty!");
-              var rtm = new RootTabletMetadata();
-              rtm.update(mutation);
-              String json = rtm.toJson();
-              log.info("Upgrading root tablet metadata, writing following to ZK : \n {}", json);
-              return json.getBytes(UTF_8);
-            });
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-
-    }
-
-  }
-
-  protected TServerInstance getLocation(ServerContext context, String relpath) {
-    String str = getFromZK(context, relpath);
-    if (str == null) {
-      return null;
-    }
-
-    String[] parts = str.split("[|]", 2);
-    HostAndPort address = HostAndPort.fromString(parts[0]);
-    if (parts.length > 1 && parts[1] != null && !parts[1].isEmpty()) {
-      return new TServerInstance(address, parts[1]);
-    } else {
-      // a 1.2 location specification: DO NOT WANT
-      return null;
-    }
-  }
-
-  static List<LogEntry> getRootLogEntries(ServerContext context) {
-
-    try {
-      ArrayList<LogEntry> result = new ArrayList<>();
-
-      ZooReaderWriter zoo = context.getZooReaderWriter();
-      String root = context.getZooKeeperRoot() + ZROOT_TABLET_WALOGS;
-      // there's a little race between getting the children and fetching
-      // the data. The log can be removed in between.
-      outer: while (true) {
-        result.clear();
-        for (String child : zoo.getChildren(root)) {
-          try {
-            @SuppressWarnings("removal")
-            LogEntry e = LogEntry.fromBytes(zoo.getData(root + "/" + child));
-            // upgrade from !0;!0<< -> +r<<
-            e = new LogEntry(RootTable.EXTENT, 0, e.filename);
-            result.add(e);
-          } catch (KeeperException.NoNodeException ex) {
-            // TODO I think this is a bug, probably meant to continue to while loop... was probably
-            // a bug in the original code.
-            continue outer;
-          }
-        }
-        break;
-      }
-
-      return result;
-    } catch (KeeperException | InterruptedException | IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private String getFromZK(ServerContext context, String relpath) {
-    try {
-      byte[] data = context.getZooReaderWriter().getData(context.getZooKeeperRoot() + relpath);
-      if (data == null) {
-        return null;
-      }
-
-      return new String(data, UTF_8);
-    } catch (NoNodeException e) {
-      return null;
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private void delete(ServerContext context, String relpath) {
-    try {
-      context.getZooReaderWriter().recursiveDelete(context.getZooKeeperRoot() + relpath,
-          NodeMissingPolicy.SKIP);
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  MetadataTime computeRootTabletTime(ServerContext context, Collection<String> goodPaths) {
-
-    try {
-      long rtime = Long.MIN_VALUE;
-      for (String good : goodPaths) {
-        Path path = new Path(good);
-
-        FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
-        var tableConf = context.getTableConfiguration(RootTable.ID);
-        long maxTime = -1;
-        try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-            .forFile(path.toString(), ns, ns.getConf(), NoCryptoServiceFactory.NONE)
-            .withTableConfiguration(tableConf).seekToBeginning().build()) {
-          while (reader.hasTop()) {
-            maxTime = Math.max(maxTime, reader.getTopKey().getTimestamp());
-            reader.next();
-          }
-        }
-        if (maxTime > rtime) {
-
-          rtime = maxTime;
-        }
-      }
-
-      if (rtime < 0) {
-        throw new IllegalStateException("Unexpected root tablet logical time " + rtime);
-      }
-
-      return new MetadataTime(rtime, TimeType.LOGICAL);
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
-
-  static Map<String,DataFileValue> cleanupRootTabletFiles(VolumeManager fs, String dir) {
-
-    try {
-      FileStatus[] files = fs.listStatus(new Path(dir));
-
-      Map<String,DataFileValue> goodFiles = new HashMap<>(files.length);
-
-      for (FileStatus file : files) {
-
-        String path = file.getPath().toString();
-        if (file.getPath().toUri().getScheme() == null) {
-          // depending on the behavior of HDFS, if list status does not return fully qualified
-          // volumes
-          // then could switch to the default volume
-          throw new IllegalArgumentException("Require fully qualified paths " + file.getPath());
-        }
-
-        String filename = file.getPath().getName();
-
-        // check for incomplete major compaction, this should only occur
-        // for root tablet
-        if (filename.startsWith("delete+")) {
-          String expectedCompactedFile =
-              path.substring(0, path.lastIndexOf("/delete+")) + "/" + filename.split("\\+")[1];
-          if (fs.exists(new Path(expectedCompactedFile))) {
-            // compaction finished, but did not finish deleting compacted files.. so delete it
-            if (!fs.deleteRecursively(file.getPath())) {
-              log.warn("Delete of file: {} return false", file.getPath());
-            }
-            continue;
-          }
-          // compaction did not finish, so put files back
-
-          // reset path and filename for rest of loop
-          filename = filename.split("\\+", 3)[2];
-          path = path.substring(0, path.lastIndexOf("/delete+")) + "/" + filename;
-          Path src = file.getPath();
-          Path dst = new Path(path);
-
-          if (!fs.rename(src, dst)) {
-            throw new IOException("Rename " + src + " to " + dst + " returned false ");
-          }
-        }
-
-        if (filename.endsWith("_tmp")) {
-          log.warn("cleaning up old tmp file: {}", path);
-          if (!fs.deleteRecursively(file.getPath())) {
-            log.warn("Delete of tmp file: {} return false", file.getPath());
-          }
-
-          continue;
-        }
-
-        if (!filename.startsWith(Constants.MAPFILE_EXTENSION + "_")
-            && !FileOperations.getValidExtensions().contains(filename.split("\\.")[1])) {
-          log.error("unknown file in tablet: {}", path);
-          continue;
-        }
-
-        goodFiles.put(path, new DataFileValue(file.getLen(), 0));
-      }
-
-      return goodFiles;
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
-
-  /**
-   * Improve how Delete markers are stored. For more information see:
-   * <a href="https://github.com/apache/accumulo/issues/1043">#1043</a>
-   * <a href="https://github.com/apache/accumulo/pull/1366">#1366</a>
-   */
-  public void upgradeFileDeletes(ServerContext context, Ample.DataLevel level) {
-
-    String tableName = level.metaTable();
-    Ample ample = context.getAmple();
-
-    // find all deletes
-    try (BatchWriter writer = context.createBatchWriter(tableName)) {
-      log.info("looking for candidates in table {}", tableName);
-      Iterator<String> oldCandidates = getOldCandidates(context, tableName);
-      String upgradeProp =
-          context.getConfiguration().get(Property.INSTANCE_VOLUMES_UPGRADE_RELATIVE);
-
-      while (oldCandidates.hasNext()) {
-        List<String> deletes = readCandidatesInBatch(oldCandidates);
-        log.info("found {} deletes to upgrade", deletes.size());
-        for (String olddelete : deletes) {
-          // create new formatted delete
-          log.trace("upgrading delete entry for {}", olddelete);
-
-          Path absolutePath = resolveRelativeDelete(olddelete, upgradeProp);
-          ReferenceFile updatedDel = switchToAllVolumes(absolutePath);
-
-          writer.addMutation(ample.createDeleteMutation(updatedDel));
-        }
-        writer.flush();
-        // if nothing thrown then we're good so mark all deleted
-        log.info("upgrade processing completed so delete old entries");
-        for (String olddelete : deletes) {
-          log.trace("deleting old entry for {}", olddelete);
-          writer.addMutation(deleteOldDeleteMutation(olddelete));
-        }
-        writer.flush();
-      }
-    } catch (TableNotFoundException | MutationsRejectedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * If path of file to delete is a directory, change it to all volumes. See {@link GcVolumeUtil}.
-   * For example: A directory "hdfs://localhost:9000/accumulo/tables/5a/t-0005" with volume removed
-   * "tables/5a/t-0005" depth = 3 will be switched to "agcav:/tables/5a/t-0005". A file
-   * "hdfs://localhost:9000/accumulo/tables/5a/t-0005/A0012.rf" with volume removed
-   * "tables/5a/t-0005/A0012.rf" depth = 4 will be returned as is.
-   */
-  @VisibleForTesting
-  static ReferenceFile switchToAllVolumes(Path olddelete) {
-    Path pathNoVolume = Objects.requireNonNull(VolumeManager.FileType.TABLE.removeVolume(olddelete),
-        "Invalid delete marker. No volume in path: " + olddelete);
-
-    // a directory path with volume removed will have a depth of 3 like, "tables/5a/t-0005"
-    if (pathNoVolume.depth() == 3) {
-      String tabletDir = pathNoVolume.getName();
-      var tableId = TableId.of(pathNoVolume.getParent().getName());
-      // except bulk directories don't get an all volume prefix
-      if (pathNoVolume.getName().startsWith(Constants.BULK_PREFIX)) {
-        return ReferenceFile.forFile(tableId, olddelete.toString());
-      } else {
-        return new AllVolumesDirectory(tableId, tabletDir);
-      }
-    } else {
-      // depth of 4 should be a file like, "tables/5a/t-0005/A0012.rf"
-      if (pathNoVolume.depth() == 4) {
-        Path tabletDirPath = pathNoVolume.getParent();
-        var tableId = TableId.of(tabletDirPath.getParent().getName());
-        return ReferenceFile.forFile(tableId, olddelete.toString());
-      } else {
-        throw new IllegalStateException("Invalid delete marker: " + olddelete);
-      }
-    }
-  }
-
-  /**
-   * Return path of the file from old delete markers
-   */
-  private Iterator<String> getOldCandidates(ServerContext context, String tableName)
-      throws TableNotFoundException {
-    Range range = DeletesSection.getRange();
-    Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY);
-    scanner.setRange(range);
-    return scanner.stream().filter(entry -> !entry.getValue().equals(UPGRADED))
-        .map(entry -> entry.getKey().getRow().toString().substring(OLD_DELETE_PREFIX.length()))
-        .iterator();
-  }
-
-  private List<String> readCandidatesInBatch(Iterator<String> candidates) {
-    long candidateLength = 0;
-    List<String> result = new ArrayList<>();
-    while (candidates.hasNext()) {
-      String candidate = candidates.next();
-      candidateLength += candidate.length();
-      result.add(candidate);
-      if (candidateLength > CANDIDATE_BATCH_SIZE) {
-        log.trace("List of delete candidates has exceeded the batch size"
-            + " threshold. Attempting to delete what has been gathered so far.");
-        break;
-      }
-    }
-    return result;
-  }
-
-  private Mutation deleteOldDeleteMutation(final String delete) {
-    Mutation m = new Mutation(OLD_DELETE_PREFIX + delete);
-    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
-    return m;
-  }
-
-  /**
-   * Changes to how volumes were stored in the metadata and have Accumulo always call the volume
-   * chooser for new tablet files. These changes were done in
-   * <a href="https://github.com/apache/accumulo/pull/1389">#1389</a>
-   */
-  public void upgradeDirColumns(ServerContext context, Ample.DataLevel level) {
-    String tableName = level.metaTable();
-
-    try (Scanner scanner = context.createScanner(tableName, Authorizations.EMPTY);
-        BatchWriter writer = context.createBatchWriter(tableName)) {
-      DIRECTORY_COLUMN.fetch(scanner);
-
-      for (Entry<Key,Value> entry : scanner) {
-        Mutation m = new Mutation(entry.getKey().getRow());
-        DIRECTORY_COLUMN.put(m, new Value(upgradeDirColumn(entry.getValue().toString())));
-        writer.addMutation(m);
-      }
-    } catch (TableNotFoundException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public static String upgradeDirColumn(String dir) {
-    return new Path(dir).getName();
-  }
-
-  /**
-   * Remove all file entries containing relative paths and replace them with absolute URI paths.
-   * Absolute paths are resolved by prefixing relative paths with a volume configured by the user in
-   * the instance.volumes.upgrade.relative property, which is only used during an upgrade. If any
-   * relative paths are found and this property is not configured, or if any resolved absolute path
-   * does not correspond to a file that actually exists, the upgrade step fails and aborts without
-   * making changes. See the property {@link Property#INSTANCE_VOLUMES_UPGRADE_RELATIVE} and the
-   * pull request <a href="https://github.com/apache/accumulo/pull/1461">#1461</a>.
-   */
-  public static void upgradeRelativePaths(ServerContext context, Ample.DataLevel level) {
-    String tableName = level.metaTable();
-    VolumeManager fs = context.getVolumeManager();
-    String upgradeProp = context.getConfiguration().get(Property.INSTANCE_VOLUMES_UPGRADE_RELATIVE);
-
-    // first pass check for relative paths - if any, check existence of the file path
-    // constructed from the upgrade property + relative path
-    if (checkForRelativePaths(context, fs, tableName, upgradeProp)) {
-      log.info("Relative Tablet File paths exist in {}, replacing with absolute using {}",
-          tableName, upgradeProp);
-    } else {
-      log.info("No relative paths found in {} during upgrade.", tableName);
-      return;
-    }
-
-    // second pass, create atomic mutations to replace the relative path
-    replaceRelativePaths(context, fs, tableName, upgradeProp);
-  }
-
-  /**
-   * Replace relative paths but only if the constructed absolute path exists on FileSystem
-   */
-  public static void replaceRelativePaths(AccumuloClient c, VolumeManager fs, String tableName,
-      String upgradeProperty) {
-    try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY);
-        BatchWriter writer = c.createBatchWriter(tableName)) {
-
-      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      for (Entry<Key,Value> entry : scanner) {
-        Key key = entry.getKey();
-        String metaEntry = key.getColumnQualifier().toString();
-        if (!metaEntry.contains(":")) {
-          // found relative paths so get the property used to build the absolute paths
-          if (upgradeProperty == null || upgradeProperty.isBlank()) {
-            throw new IllegalArgumentException(
-                "Missing required property " + Property.INSTANCE_VOLUMES_UPGRADE_RELATIVE.getKey());
-          }
-          Path relPath = resolveRelativePath(metaEntry, key);
-          Path absPath = new Path(upgradeProperty, relPath);
-          if (fs.exists(absPath)) {
-            log.debug("Changing Tablet File path from {} to {}", metaEntry, absPath);
-            Mutation m = new Mutation(key.getRow());
-            // add the new path
-            m.at().family(key.getColumnFamily()).qualifier(absPath.toString())
-                .visibility(key.getColumnVisibility()).put(entry.getValue());
-            // delete the old path
-            m.at().family(key.getColumnFamily()).qualifier(key.getColumnQualifierData().toArray())
-                .visibility(key.getColumnVisibility()).delete();
-            writer.addMutation(m);
-          } else {
-            throw new IllegalArgumentException(
-                "Relative Tablet file " + relPath + " not found at " + absPath);
-          }
-        }
-      }
-    } catch (MutationsRejectedException | TableNotFoundException e) {
-      throw new IllegalStateException(e);
-    } catch (IOException ioe) {
-      throw new UncheckedIOException(ioe);
-    }
-  }
-
-  /**
-   * Check if table has any relative paths, return false if none are found. When a relative path is
-   * found, check existence of the file path constructed from the upgrade property + relative path
-   */
-  public static boolean checkForRelativePaths(AccumuloClient client, VolumeManager fs,
-      String tableName, String upgradeProperty) {
-    boolean hasRelatives = false;
-
-    try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
-      log.info("Looking for relative paths in {}", tableName);
-      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      for (Entry<Key,Value> entry : scanner) {
-        Key key = entry.getKey();
-        String metaEntry = key.getColumnQualifier().toString();
-        if (!metaEntry.contains(":")) {
-          // found relative paths so verify the property used to build the absolute paths
-          hasRelatives = true;
-          if (upgradeProperty == null || upgradeProperty.isBlank()) {
-            throw new IllegalArgumentException(
-                "Missing required property " + Property.INSTANCE_VOLUMES_UPGRADE_RELATIVE.getKey());
-          }
-          Path relPath = resolveRelativePath(metaEntry, key);
-          Path absPath = new Path(upgradeProperty, relPath);
-          if (!fs.exists(absPath)) {
-            throw new IllegalArgumentException("Tablet file " + relPath + " not found at " + absPath
-                + " using volume: " + upgradeProperty);
-          }
-        }
-      }
-    } catch (TableNotFoundException e) {
-      throw new IllegalStateException(e);
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-
-    return hasRelatives;
-  }
-
-  /**
-   * Resolve old-style relative paths, returning Path of everything except volume and base
-   */
-  private static Path resolveRelativePath(String metadataEntry, Key key) {
-    String prefix = VolumeManager.FileType.TABLE.getDirectory() + "/";
-    if (metadataEntry.startsWith("../")) {
-      // resolve style "../2a/t-0003/C0004.rf"
-      return new Path(prefix + metadataEntry.substring(3));
-    } else {
-      // resolve style "/t-0003/C0004.rf"
-      TableId tableId = KeyExtent.fromMetaRow(key.getRow()).tableId();
-      return new Path(prefix + tableId.canonical() + metadataEntry);
-    }
-  }
-
-  /**
-   * Resolve old relative delete markers of the form /tableId/tabletDir/[file] to
-   * UpgradeVolume/tables/tableId/tabletDir/[file]
-   */
-  static Path resolveRelativeDelete(String oldDelete, String upgradeProperty) {
-    Path pathNoVolume = VolumeManager.FileType.TABLE.removeVolume(new Path(oldDelete));
-    Path pathToCheck = new Path(oldDelete);
-
-    // if the volume was removed properly, the path is absolute so return, otherwise
-    // it is a relative path so proceed with more checks
-    if (pathNoVolume != null) {
-      return pathToCheck;
-    }
-
-    // A relative path directory of the form "/tableId/tabletDir" will have depth == 2
-    // A relative path file of the form "/tableId/tabletDir/file" will have depth == 3
-    Preconditions.checkState(
-        oldDelete.startsWith("/") && (pathToCheck.depth() == 2 || pathToCheck.depth() == 3),
-        "Unrecognized relative delete marker (%s)", oldDelete);
-
-    // found relative paths so verify the property used to build the absolute paths
-    if (upgradeProperty == null || upgradeProperty.isBlank()) {
-      throw new IllegalArgumentException(
-          "Missing required property " + Property.INSTANCE_VOLUMES_UPGRADE_RELATIVE.getKey());
-    }
-    return new Path(upgradeProperty, VolumeManager.FileType.TABLE.getDirectory() + oldDelete);
-  }
-
-  /**
-   * Remove old temporary map files to prevent problems during recovery. Sorted recovery was updated
-   * to use RFiles instead of map files. So to prevent issues during tablet recovery, remove the old
-   * temporary map files and resort using RFiles. For more information see the following issues:
-   * <a href="https://github.com/apache/accumulo/issues/2117">#2117</a> and
-   * <a href="https://github.com/apache/accumulo/issues/2179">#2179</a>
-   */
-  static void dropSortedMapWALFiles(ServerContext context) {
-    VolumeManager vm = context.getVolumeManager();
-    for (String recoveryDir : context.getRecoveryDirs()) {
-      Path recoveryDirPath = new Path(recoveryDir);
-      try {
-        if (!vm.exists(recoveryDirPath)) {
-          log.info("There are no recovery files in {}", recoveryDir);
-          continue;
-        }
-        List<Path> directoriesToDrop = new ArrayList<>();
-        for (FileStatus walDir : vm.listStatus(recoveryDirPath)) {
-          // map files will be in a directory starting with "part"
-          Path walDirPath = walDir.getPath();
-          for (FileStatus dirOrFile : vm.listStatus(walDirPath)) {
-            if (dirOrFile.isDirectory()) {
-              directoriesToDrop.add(walDirPath);
-              break;
-            }
-          }
-        }
-        if (!directoriesToDrop.isEmpty()) {
-          log.info("Found {} old sorted map directories to delete.", directoriesToDrop.size());
-          for (Path dir : directoriesToDrop) {
-            log.info("Deleting everything in old sorted map directory: {}", dir);
-            vm.deleteRecursively(dir);
-          }
-        }
-      } catch (IOException ioe) {
-        throw new UncheckedIOException(ioe);
-      }
-    }
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/Master.java b/server/manager/src/main/java/org/apache/accumulo/master/Master.java
deleted file mode 100644
index 4005168..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/master/Master.java
+++ /dev/null
@@ -1,38 +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.master;
-
-import org.apache.accumulo.manager.MasterExecutable;
-import org.apache.accumulo.start.Main;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 2.1.0. Use {@link Main} with keyword "manager" instead.
- */
-@Deprecated(since = "2.1.0")
-public class Master {
-  private static final Logger LOG = LoggerFactory.getLogger(Master.class);
-
-  public static void main(String[] args) throws Exception {
-    LOG.warn("Usage of {} directly has been deprecated. Use {} with keyword manager instead.",
-        Master.class.getName(), Main.class.getName());
-    new MasterExecutable().execute(args);
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/replication/UnorderedWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/master/replication/UnorderedWorkAssigner.java
deleted file mode 100644
index a25b1f7..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/master/replication/UnorderedWorkAssigner.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.master.replication;
-
-import org.apache.accumulo.core.conf.Property;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * @deprecated since 2.1.0. Use
- *             {@link org.apache.accumulo.manager.replication.UnorderedWorkAssigner} instead
- */
-@Deprecated(since = "2.1.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Compatibility class to be removed in next major release.")
-public class UnorderedWorkAssigner
-    extends org.apache.accumulo.manager.replication.UnorderedWorkAssigner {
-  private static final Logger log = LoggerFactory.getLogger(UnorderedWorkAssigner.class);
-
-  public UnorderedWorkAssigner() {
-    log.warn("{} has been deprecated. Please update property {} to {} instead.",
-        getClass().getName(), Property.REPLICATION_WORK_ASSIGNER.getKey(),
-        org.apache.accumulo.manager.replication.UnorderedWorkAssigner.class.getName());
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/state/MergeStats.java b/server/manager/src/main/java/org/apache/accumulo/master/state/MergeStats.java
deleted file mode 100644
index 006a33c..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ /dev/null
@@ -1,36 +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.master.state;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.manager.state.MergeStats} instead.
- */
-@Deprecated(since = "2.1.0")
-public class MergeStats {
-  final static private Logger log = LoggerFactory.getLogger(MergeStats.class);
-
-  public static void main(String[] args) throws Exception {
-    log.warn("{} is deprecated. Use {} instead.", MergeStats.class.getName(),
-        org.apache.accumulo.manager.state.MergeStats.class.getName());
-    org.apache.accumulo.manager.state.MergeStats.main(args);
-  }
-}
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/state/SetGoalState.java b/server/manager/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
deleted file mode 100644
index fd9a997..0000000
--- a/server/manager/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
+++ /dev/null
@@ -1,39 +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.master.state;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * @deprecated since 2.1.0. Use {@link org.apache.accumulo.manager.state.SetGoalState} instead.
- */
-@Deprecated(since = "2.1.0")
-public class SetGoalState {
-  final static private Logger log = LoggerFactory.getLogger(SetGoalState.class);
-
-  /**
-   * Utility program that will change the goal state for the master from the command line.
-   */
-  public static void main(String[] args) throws Exception {
-    log.warn("{} is deprecated. Use {} instead.", SetGoalState.class.getName(),
-        org.apache.accumulo.manager.state.SetGoalState.class.getName());
-    org.apache.accumulo.manager.state.SetGoalState.main(args);
-  }
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/ManagerTimeTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/ManagerTimeTest.java
new file mode 100644
index 0000000..528eccc
--- /dev/null
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/ManagerTimeTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.manager;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.util.time.SteadyTime;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+public class ManagerTimeTest {
+
+  @Test
+  public void testSteadyTime() {
+    long time = 20_000;
+    var steadyTime = SteadyTime.from(time, TimeUnit.NANOSECONDS);
+
+    // make sure calling serialize on instance matches static helper
+    byte[] serialized = ManagerTime.serialize(steadyTime);
+    assertArrayEquals(serialized, ManagerTime.serialize(steadyTime));
+
+    // Verify deserialization matches original object
+    var deserialized = ManagerTime.deserialize(serialized);
+    assertEquals(steadyTime, deserialized);
+    assertEquals(0, steadyTime.compareTo(deserialized));
+  }
+
+  @ParameterizedTest
+  // Test with both a 0 and positive previous value. This simulates the value
+  // read out of zookeeper for the time and should never be negative as it is
+  // based on elapsed time from previous manager runs
+  @ValueSource(longs = {0, 50_000})
+  public void testSteadyTimeFromSkew(long previousTime) throws InterruptedException {
+    List<Long> times = List.of(-100_000L, -100L, 0L, 20_000L, System.nanoTime());
+
+    for (Long time : times) {
+      // ManagerTime builds the skew amount by subtracting the current nanotime
+      // from the previous persisted time in ZK. The skew can be negative or positive because
+      // it will depend on if the current nanotime is negative or positive as
+      // nanotime is allowed to be negative
+      var skewAmount =
+          ManagerTime.updateSkew(SteadyTime.from(previousTime, TimeUnit.NANOSECONDS), time);
+
+      // Build a SteadyTime using the skewAmount
+      // SteadyTime should never be negative
+      var original = ManagerTime.fromSkew(time, skewAmount);
+
+      // Simulate a future time and create another SteadyTime from the skew which should
+      // now be after the original
+      time = time + 10000;
+      var futureSkew = ManagerTime.fromSkew(time, skewAmount);
+
+      // future should be after the original
+      assertTrue(futureSkew.compareTo(original) > 0);
+    }
+  }
+
+  @ParameterizedTest
+  // Test with both a 0 and positive previous value. This simulates the value
+  // read out of zookeeper for the time and should never be negative as it is
+  // based on elapsed time from previous manager runs
+  @ValueSource(longs = {0, 50_000})
+  public void testSteadyTimeFromSkewCurrent(long previousTime) throws InterruptedException {
+    // Also test fromSkew(skewAmount) method which only uses System.nanoTime()
+    var skewAmount = ManagerTime.updateSkew(SteadyTime.from(previousTime, TimeUnit.NANOSECONDS));
+
+    // Build a SteadyTime using the skewAmount and current time
+    var original = ManagerTime.fromSkew(skewAmount);
+
+    // sleep a bit so time elapses
+    Thread.sleep(10);
+    var futureSkew = ManagerTime.fromSkew(skewAmount);
+
+    // future should be after the original
+    assertTrue(futureSkew.compareTo(original) > 0);
+  }
+
+  @ParameterizedTest
+  // Test with both a 0 and positive previous value. This simulates the value
+  // read out of zookeeper for the time and should never be negative as it is
+  // based on elapsed time from previous manager runs
+  @ValueSource(longs = {0, 50_000})
+  public void testSteadyTimeUpdateSkew(long previousTime) throws InterruptedException {
+
+    var steadyTime = SteadyTime.from(previousTime, TimeUnit.NANOSECONDS);
+    List<Long> times = List.of(-100_000L, -100L, 0L, 20_000L, System.nanoTime());
+
+    // test updateSkew with various times and previous times
+    for (Long time : times) {
+      var expected = steadyTime.getNanos() - time;
+
+      // test that updateSkew computes the update as current steadyTime - time
+      var skewAmount = ManagerTime.updateSkew(steadyTime, time);
+      assertEquals(expected, skewAmount.toNanos());
+    }
+
+    // test updateSkew with current system time
+    var skew = ManagerTime.updateSkew(steadyTime);
+    // sleep a bit so time elapses
+    Thread.sleep(10);
+    var updatedSkew = ManagerTime.updateSkew(steadyTime);
+    // Updating the skew time subtracts the current nanotime from
+    // the previous value so the updated value should be less than
+    // a previously created SteadyTime based on the same skew
+    assertTrue(skew.toNanos() - updatedSkew.toNanos() > 0);
+    assertTrue(skew.compareTo(updatedSkew) > 0);
+  }
+}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java
new file mode 100644
index 0000000..b601d2b
--- /dev/null
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.manager;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.manager.TabletGroupWatcher.HighTablet;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+
+public class TabletGroupWatcherTest {
+
+  @Test
+  public void testComputeNewDfvEven() {
+    DataFileValue original = new DataFileValue(20, 10, 100);
+    Pair<DataFileValue,DataFileValue> newValues = TabletGroupWatcher.computeNewDfv(original);
+
+    assertEquals(10, newValues.getFirst().getSize());
+    assertEquals(5, newValues.getFirst().getNumEntries());
+    assertEquals(original.getTime(), newValues.getFirst().getTime());
+    assertEquals(10, newValues.getSecond().getSize());
+    assertEquals(5, newValues.getSecond().getNumEntries());
+    assertEquals(original.getTime(), newValues.getSecond().getTime());
+  }
+
+  @Test
+  public void testComputeNewDfvOdd() {
+    DataFileValue original = new DataFileValue(21, 11, 100);
+    Pair<DataFileValue,DataFileValue> newValues = TabletGroupWatcher.computeNewDfv(original);
+
+    assertEquals(10, newValues.getFirst().getSize());
+    assertEquals(5, newValues.getFirst().getNumEntries());
+    assertEquals(original.getTime(), newValues.getFirst().getTime());
+    assertEquals(11, newValues.getSecond().getSize());
+    assertEquals(6, newValues.getSecond().getNumEntries());
+    assertEquals(original.getTime(), newValues.getSecond().getTime());
+  }
+
+  @Test
+  public void testComputeNewDfvSmall() {
+    DataFileValue original = new DataFileValue(1, 2, 100);
+    Pair<DataFileValue,DataFileValue> newValues = TabletGroupWatcher.computeNewDfv(original);
+
+    assertEquals(1, newValues.getFirst().getSize());
+    assertEquals(1, newValues.getFirst().getNumEntries());
+    assertEquals(original.getTime(), newValues.getFirst().getTime());
+    assertEquals(1, newValues.getSecond().getSize());
+    assertEquals(1, newValues.getSecond().getNumEntries());
+    assertEquals(original.getTime(), newValues.getSecond().getTime());
+  }
+
+  @Test
+  public void testHighTablet() {
+    HighTablet mergedTruePrevRowFalse = new HighTablet(
+        new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("end"), null), true);
+    assertNotNull(mergedTruePrevRowFalse.getExtent());
+    assertTrue(mergedTruePrevRowFalse.isMerged());
+
+    HighTablet mergedFalsePrevRowFalse = new HighTablet(
+        new KeyExtent(AccumuloTable.METADATA.tableId(), new Text("end"), null), false);
+    assertNotNull(mergedFalsePrevRowFalse.getExtent());
+    assertFalse(mergedFalsePrevRowFalse.isMerged());
+  }
+}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.java
deleted file mode 100644
index de9755e..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.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.manager.metrics;
-
-import java.lang.reflect.Field;
-import java.time.Duration;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.replication.ReplicationUtil;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.Test;
-
-import io.micrometer.core.instrument.MeterRegistry;
-import io.micrometer.core.instrument.Timer;
-
-@Deprecated
-public class ReplicationMetricsTest {
-
-  /**
-   * Extend the class to override the current time for testing
-   */
-  public class ReplicationMetricsTestMetrics extends ReplicationMetrics {
-    ReplicationMetricsTestMetrics(Manager manager) {
-      super(manager);
-    }
-
-  }
-
-  @Test
-  public void testAddReplicationQueueTimeMetrics() throws Exception {
-    Manager manager = EasyMock.createMock(Manager.class);
-    ServerContext context = EasyMock.createMock(ServerContext.class);
-    VolumeManager fileSystem = EasyMock.createMock(VolumeManager.class);
-    ReplicationUtil util = EasyMock.createMock(ReplicationUtil.class);
-    MeterRegistry meterRegistry = EasyMock.createMock(MeterRegistry.class);
-    Timer timer = EasyMock.createMock(Timer.class);
-
-    Path path1 = new Path("hdfs://localhost:9000/accumulo/wal/file1");
-    Path path2 = new Path("hdfs://localhost:9000/accumulo/wal/file2");
-
-    // First call will initialize the map of paths to modification time
-    EasyMock.expect(manager.getContext()).andReturn(context).anyTimes();
-    EasyMock.expect(meterRegistry.timer("replicationQueue")).andReturn(timer).anyTimes();
-    EasyMock.expect(meterRegistry.gauge(EasyMock.eq("filesPendingReplication"),
-        EasyMock.anyObject(AtomicLong.class))).andReturn(new AtomicLong(0)).anyTimes();
-    EasyMock
-        .expect(
-            meterRegistry.gauge(EasyMock.eq("numPeers"), EasyMock.anyObject(AtomicInteger.class)))
-        .andReturn(new AtomicInteger(0)).anyTimes();
-    EasyMock.expect(meterRegistry.gauge(EasyMock.eq("maxReplicationThreads"),
-        EasyMock.anyObject(AtomicInteger.class))).andReturn(new AtomicInteger(0)).anyTimes();
-    EasyMock.expect(util.getPendingReplicationPaths()).andReturn(Set.of(path1, path2));
-    EasyMock.expect(manager.getVolumeManager()).andReturn(fileSystem);
-    EasyMock.expect(fileSystem.getFileStatus(path1)).andReturn(createStatus(100));
-    EasyMock.expect(manager.getVolumeManager()).andReturn(fileSystem);
-    EasyMock.expect(fileSystem.getFileStatus(path2)).andReturn(createStatus(200));
-
-    // Second call will recognize the missing path1 and add the latency stat
-    EasyMock.expect(util.getPendingReplicationPaths()).andReturn(Set.of(path2));
-
-    timer.record(EasyMock.isA(Duration.class));
-    EasyMock.expectLastCall();
-
-    EasyMock.replay(manager, fileSystem, util, meterRegistry, timer);
-
-    ReplicationMetrics metrics = new ReplicationMetricsTestMetrics(manager);
-
-    // Inject our mock objects
-    replaceField(metrics, "replicationUtil", util);
-    replaceField(metrics, "replicationQueueTimer", timer);
-
-    // Two calls to this will initialize the map and then add metrics
-    metrics.addReplicationQueueTimeMetrics();
-    metrics.addReplicationQueueTimeMetrics();
-
-    EasyMock.verify(manager, fileSystem, util, meterRegistry, timer);
-  }
-
-  private void replaceField(Object instance, String fieldName, Object target)
-      throws NoSuchFieldException, IllegalAccessException {
-    Field field = instance.getClass().getSuperclass().getDeclaredField(fieldName);
-    field.setAccessible(true);
-    field.set(instance, target);
-  }
-
-  private FileStatus createStatus(long modtime) {
-    return new FileStatus(0, false, 0, 0, modtime, 0, null, null, null, null);
-  }
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssignerHelperTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssignerHelperTest.java
deleted file mode 100644
index 1c7f43d..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/replication/DistributedWorkQueueWorkAssignerHelperTest.java
+++ /dev/null
@@ -1,59 +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.manager.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Map.Entry;
-import java.util.UUID;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.hadoop.fs.Path;
-import org.apache.zookeeper.common.PathUtils;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class DistributedWorkQueueWorkAssignerHelperTest {
-
-  @Test
-  public void createsValidZKNodeName() {
-    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID());
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-
-    String key = DistributedWorkQueueWorkAssignerHelper.getQueueKey(p.toString(), target);
-
-    PathUtils.validatePath(key);
-  }
-
-  @Test
-  public void queueKeySerialization() {
-    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID());
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-
-    String key = DistributedWorkQueueWorkAssignerHelper.getQueueKey(p.toString(), target);
-
-    Entry<String,ReplicationTarget> result =
-        DistributedWorkQueueWorkAssignerHelper.fromQueueKey(key);
-    assertEquals(p.toString(), result.getKey());
-    assertEquals(target, result.getValue());
-  }
-
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.java
deleted file mode 100644
index 45afcde..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.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.manager.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import java.util.Collections;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReader;
-import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.server.ServerContext;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class ManagerReplicationCoordinatorTest {
-
-  static AccumuloConfiguration config = DefaultConfiguration.getInstance();
-
-  @Test
-  public void randomServer() {
-    Manager manager = createMock(Manager.class);
-    ZooReader reader = createMock(ZooReader.class);
-    ServerContext context = createMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(config).anyTimes();
-    expect(context.getInstanceID()).andReturn(InstanceId.of("1234")).anyTimes();
-    expect(context.getZooReader()).andReturn(null).anyTimes();
-    expect(context.getZooReaderWriter()).andReturn(null).anyTimes();
-    expect(context.getSecurityOperation()).andReturn(null).anyTimes();
-    expect(manager.getContext()).andReturn(context);
-    expect(manager.getInstanceID()).andReturn(InstanceId.of("1234"));
-    replay(manager, context, reader);
-
-    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(manager, reader);
-    TServerInstance inst1 = new TServerInstance(HostAndPort.fromParts("host1", 1234), "session");
-
-    assertEquals(inst1, coordinator.getRandomTServer(Collections.singleton(inst1), 0));
-  }
-
-  @Test
-  public void invalidOffset() {
-    Manager manager = createMock(Manager.class);
-    ServerContext context = createMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(config).anyTimes();
-    expect(context.getInstanceID()).andReturn(InstanceId.of("1234")).anyTimes();
-    expect(context.getZooReader()).andReturn(null).anyTimes();
-    expect(context.getZooReaderWriter()).andReturn(null).anyTimes();
-    expect(context.getSecurityOperation()).andReturn(null).anyTimes();
-    ZooReader reader = createMock(ZooReader.class);
-    expect(manager.getContext()).andReturn(context);
-    expect(manager.getInstanceID()).andReturn(InstanceId.of("1234"));
-    replay(manager, context, reader);
-    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(manager, reader);
-    TServerInstance inst1 = new TServerInstance(HostAndPort.fromParts("host1", 1234), "session");
-    assertThrows(IllegalArgumentException.class,
-        () -> coordinator.getRandomTServer(Collections.singleton(inst1), 1));
-  }
-
-  @Test
-  public void randomServerFromMany() {
-    Manager manager = createMock(Manager.class);
-    ZooReader reader = createMock(ZooReader.class);
-    ServerContext context = createMock(ServerContext.class);
-    expect(context.getConfiguration()).andReturn(config).anyTimes();
-    expect(context.getInstanceID()).andReturn(InstanceId.of("1234")).anyTimes();
-    expect(context.getZooReader()).andReturn(null).anyTimes();
-    expect(context.getZooReaderWriter()).andReturn(null).anyTimes();
-    expect(context.getSecurityOperation()).andReturn(null).anyTimes();
-    expect(manager.getInstanceID()).andReturn(InstanceId.of("1234")).anyTimes();
-    expect(manager.getContext()).andReturn(context).anyTimes();
-    replay(manager, context, reader);
-
-    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(manager, reader);
-
-    verify(manager, reader);
-
-    TreeSet<TServerInstance> instances = new TreeSet<>();
-    TServerInstance inst1 = new TServerInstance(HostAndPort.fromParts("host1", 1234), "session");
-    instances.add(inst1);
-    TServerInstance inst2 = new TServerInstance(HostAndPort.fromParts("host2", 1234), "session");
-    instances.add(inst2);
-
-    assertEquals(inst1, coordinator.getRandomTServer(instances, 0));
-    assertEquals(inst2, coordinator.getRandomTServer(instances, 1));
-  }
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/replication/SequentialWorkAssignerTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/replication/SequentialWorkAssignerTest.java
deleted file mode 100644
index 264556b..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/replication/SequentialWorkAssignerTest.java
+++ /dev/null
@@ -1,109 +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.manager.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class SequentialWorkAssignerTest {
-
-  private AccumuloClient client;
-  private SequentialWorkAssigner assigner;
-
-  @BeforeEach
-  public void init() {
-    AccumuloConfiguration conf = createMock(AccumuloConfiguration.class);
-    client = createMock(AccumuloClient.class);
-    assigner = new SequentialWorkAssigner(conf, client);
-  }
-
-  @Test
-  public void basicZooKeeperCleanup() {
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    ZooCache zooCache = createMock(ZooCache.class);
-
-    Map<String,Map<TableId,String>> queuedWork = new TreeMap<>();
-    Map<TableId,String> cluster1Work = new TreeMap<>();
-
-    // Two files for cluster1, one for table '1' and another for table '2' we haven't assigned work
-    // for
-    cluster1Work.put(TableId.of("1"), DistributedWorkQueueWorkAssignerHelper.getQueueKey("file1",
-        new ReplicationTarget("cluster1", "1", TableId.of("1"))));
-    cluster1Work.put(TableId.of("2"), DistributedWorkQueueWorkAssignerHelper.getQueueKey("file2",
-        new ReplicationTarget("cluster1", "2", TableId.of("2"))));
-
-    queuedWork.put("cluster1", cluster1Work);
-
-    assigner.setClient(client);
-    assigner.setZooCache(zooCache);
-    assigner.setWorkQueue(workQueue);
-    assigner.setQueuedWork(queuedWork);
-
-    InstanceOperations opts = createMock(InstanceOperations.class);
-    var iid = InstanceId.of("instance");
-    expect(opts.getInstanceId()).andReturn(iid);
-    expect(client.instanceOperations()).andReturn(opts);
-
-    // file1 replicated
-    expect(zooCache.get(ZooUtil.getRoot(iid) + ReplicationConstants.ZOO_WORK_QUEUE + "/"
-        + DistributedWorkQueueWorkAssignerHelper.getQueueKey("file1",
-            new ReplicationTarget("cluster1", "1", TableId.of("1")))))
-        .andReturn(null);
-    // file2 still needs to replicate
-    expect(
-        zooCache
-            .get(ZooUtil.getRoot(iid) + ReplicationConstants.ZOO_WORK_QUEUE + "/"
-                + DistributedWorkQueueWorkAssignerHelper.getQueueKey("file2",
-                    new ReplicationTarget("cluster1", "2", TableId.of("2")))))
-        .andReturn(new byte[0]);
-
-    replay(workQueue, zooCache, opts, client);
-
-    assigner.cleanupFinishedWork();
-
-    verify(workQueue, zooCache, client);
-
-    assertEquals(1, cluster1Work.size());
-    assertEquals(
-        DistributedWorkQueueWorkAssignerHelper.getQueueKey("file2",
-            new ReplicationTarget("cluster1", "2", TableId.of("2"))),
-        cluster1Work.get(TableId.of("2")));
-  }
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/replication/UnorderedWorkAssignerTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/replication/UnorderedWorkAssignerTest.java
deleted file mode 100644
index 7b1af35..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/replication/UnorderedWorkAssignerTest.java
+++ /dev/null
@@ -1,138 +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.manager.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.hadoop.fs.Path;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-@Disabled("Replication Tests are not stable and not currently maintained")
-public class UnorderedWorkAssignerTest {
-
-  private AccumuloClient client;
-  private UnorderedWorkAssigner assigner;
-
-  @BeforeEach
-  public void init() {
-    AccumuloConfiguration conf = createMock(AccumuloConfiguration.class);
-    client = createMock(AccumuloClient.class);
-    assigner = new UnorderedWorkAssigner(conf, client);
-  }
-
-  @Test
-  public void workQueuedUsingFileName() throws Exception {
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Set<String> queuedWork = new HashSet<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-
-    Path p = new Path("/accumulo/wal/tserver+port/" + UUID.randomUUID());
-
-    String expectedQueueKey =
-        p.getName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getPeerName()
-            + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
-            + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
-
-    workQueue.addWork(expectedQueueKey, p.toString());
-    expectLastCall().once();
-
-    replay(workQueue);
-
-    assigner.queueWork(p, target);
-
-    assertEquals(1, queuedWork.size());
-    assertEquals(expectedQueueKey, queuedWork.iterator().next());
-  }
-
-  @Test
-  public void existingWorkIsReQueued() throws Exception {
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-
-    List<String> existingWork =
-        Arrays.asList("/accumulo/wal/tserver+port/wal1", "/accumulo/wal/tserver+port/wal2");
-    expect(workQueue.getWorkQueued()).andReturn(existingWork);
-
-    replay(workQueue);
-
-    assigner.setWorkQueue(workQueue);
-    assigner.initializeQueuedWork();
-
-    verify(workQueue);
-
-    Set<String> queuedWork = assigner.getQueuedWork();
-    assertEquals(existingWork.size(), queuedWork.size(),
-        "Expected existing work and queued work to be the same size");
-    assertTrue(queuedWork.containsAll(existingWork), "Expected all existing work to be queued");
-  }
-
-  @Test
-  public void workNotInZooKeeperIsCleanedUp() {
-    Set<String> queuedWork = new LinkedHashSet<>(Arrays.asList("wal1", "wal2"));
-    assigner.setQueuedWork(queuedWork);
-
-    ZooCache cache = createMock(ZooCache.class);
-    assigner.setZooCache(cache);
-
-    InstanceOperations opts = createMock(InstanceOperations.class);
-    expect(opts.getInstanceID()).andReturn("id");
-    expect(client.instanceOperations()).andReturn(opts);
-
-    expect(cache.get(Constants.ZROOT + "/id" + ReplicationConstants.ZOO_WORK_QUEUE + "/wal1"))
-        .andReturn(null);
-    expect(cache.get(Constants.ZROOT + "/id" + ReplicationConstants.ZOO_WORK_QUEUE + "/wal2"))
-        .andReturn(null);
-
-    replay(cache, opts, client);
-
-    assigner.cleanupFinishedWork();
-
-    verify(cache, client);
-    assertTrue(queuedWork.isEmpty(), "Queued work was not emptied");
-  }
-
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/replication/WorkMakerTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/replication/WorkMakerTest.java
deleted file mode 100644
index 92a0eb2..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/replication/WorkMakerTest.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.manager.replication;
-
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class WorkMakerTest {
-
-  @Test
-  public void closedStatusRecordsStillMakeWork() {
-    WorkMaker workMaker = new WorkMaker(null, null);
-
-    assertFalse(workMaker.shouldCreateWork(StatusUtil.fileCreated(System.currentTimeMillis())));
-    assertTrue(workMaker.shouldCreateWork(StatusUtil.ingestedUntil(1000)));
-    // We don't need to re-create work for something that's already replicated.
-    assertFalse(workMaker.shouldCreateWork(Status.newBuilder().setBegin(Long.MAX_VALUE).setEnd(0)
-        .setInfiniteEnd(true).setClosed(true).build()));
-  }
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/state/MergeStatsTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/state/MergeStatsTest.java
index c9ad976..b1663ad 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/state/MergeStatsTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/state/MergeStatsTest.java
@@ -24,11 +24,13 @@
 
 import java.util.Collection;
 import java.util.List;
+import java.util.UUID;
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletLocationState.BadLocationStateException;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.manager.state.MergeInfo;
 import org.apache.accumulo.server.manager.state.MergeInfo.Operation;
 import org.apache.accumulo.server.manager.state.MergeState;
@@ -48,7 +50,7 @@
     stats.verifyState(mergeInfo, MergeState.WAITING_FOR_OFFLINE);
 
     // State is wrong so should throw exception
-    mergeInfo.setState(MergeState.WAITING_FOR_CHOPPED);
+    mergeInfo.setState(MergeState.MERGING);
     assertThrows(IllegalStateException.class,
         () -> stats.verifyState(mergeInfo, MergeState.WAITING_FOR_OFFLINE));
   }
@@ -60,12 +62,13 @@
 
     // Verify that if there are Walogs the return true, else false
     assertTrue(stats.verifyWalogs(getState(keyExtent, List.of())));
-    assertFalse(stats.verifyWalogs(getState(keyExtent, List.of(List.of("log1")))));
+    LogEntry log1 = LogEntry.fromPath("file:///dir/tserver+9997/" + UUID.randomUUID());
+    assertFalse(stats.verifyWalogs(getState(keyExtent, List.of(log1))));
   }
 
-  private TabletLocationState getState(KeyExtent keyExtent, Collection<Collection<String>> walogs)
+  private TabletLocationState getState(KeyExtent keyExtent, Collection<LogEntry> walogs)
       throws BadLocationStateException {
-    return new TabletLocationState(keyExtent, null, null, null, null, walogs, true);
+    return new TabletLocationState(keyExtent, null, null, null, null, walogs);
   }
 
 }
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java
index 8384a4a..9aff9a8 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java
@@ -26,16 +26,17 @@
 import java.util.HashMap;
 
 import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.TServerInstance;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tserverOps.ShutdownTServer;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
 import org.easymock.EasyMock;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class ShutdownTServerTest {
 
   @Test
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImportTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImportTest.java
index 62bbbd1..dff9098 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImportTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImportTest.java
@@ -31,6 +31,7 @@
 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.Optional;
@@ -96,21 +97,29 @@
 
   public void runTest(Map<KeyExtent,String> loadRanges, List<KeyExtent> tabletRanges,
       int maxTablets) throws Exception {
-    TabletIterFactory tabletIterFactory = startRow -> {
-      int start = -1;
+    TabletIterFactory tabletIterFactory = new TabletIterFactory() {
+      @Override
+      public Iterator<KeyExtent> newTabletIter(Text startRow) {
+        int start = -1;
 
-      if (startRow == null) {
-        start = 0;
-      } else {
-        for (int i = 0; i < tabletRanges.size(); i++) {
-          if (tabletRanges.get(i).contains(startRow)) {
-            start = i;
-            break;
+        if (startRow == null) {
+          start = 0;
+        } else {
+          for (int i = 0; i < tabletRanges.size(); i++) {
+            if (tabletRanges.get(i).contains(startRow)) {
+              start = i;
+              break;
+            }
           }
         }
+
+        return tabletRanges.subList(start, tabletRanges.size()).iterator();
       }
 
-      return tabletRanges.subList(start, tabletRanges.size()).iterator();
+      @Override
+      public void close() {
+        // nothing to close
+      }
     };
 
     var sortedExtents = loadRanges.keySet().stream().sorted().collect(Collectors.toList());
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
deleted file mode 100644
index 638123b..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
+++ /dev/null
@@ -1,205 +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.manager.upgrade;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
-import org.apache.accumulo.manager.WithTestNames;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.hadoop.fs.Path;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-public class RootFilesUpgradeTest extends WithTestNames {
-
-  @TempDir
-  private static File tempDir;
-
-  static void rename(VolumeManager fs, Path src, Path dst) throws IOException {
-    if (!fs.rename(src, dst)) {
-      throw new IOException("Rename " + src + " to " + dst + " returned false ");
-    }
-  }
-
-  private class TestWrapper {
-    File rootTabletDir;
-    Set<Path> oldDatafiles;
-    String compactName;
-    Path tmpDatafile;
-    Path newDatafile;
-    VolumeManager vm;
-    AccumuloConfiguration conf;
-
-    public void prepareReplacement(VolumeManager fs, Path location, Set<Path> oldDatafiles,
-        String compactName) throws IOException {
-      for (Path path : oldDatafiles) {
-        rename(fs, path, new Path(location + "/delete+" + compactName + "+" + path.getName()));
-      }
-    }
-
-    public void renameReplacement(VolumeManager fs, Path tmpDatafile, Path newDatafile)
-        throws IOException {
-      if (fs.exists(newDatafile)) {
-        throw new IllegalStateException("Target map file already exist " + newDatafile);
-      }
-
-      rename(fs, tmpDatafile, newDatafile);
-    }
-
-    public void finishReplacement(AccumuloConfiguration acuTableConf, VolumeManager fs,
-        Path location, Set<Path> oldDatafiles, String compactName) throws IOException {
-      // start deleting files, if we do not finish they will be cleaned
-      // up later
-      for (Path path : oldDatafiles) {
-        Path deleteFile = new Path(location + "/delete+" + compactName + "+" + path.getName());
-        @SuppressWarnings("removal")
-        Property p = Property.GC_TRASH_IGNORE;
-        if (acuTableConf.getBoolean(p) || !fs.moveToTrash(deleteFile)) {
-          fs.deleteRecursively(deleteFile);
-        }
-      }
-    }
-
-    TestWrapper(VolumeManager vm, AccumuloConfiguration conf, String dirName, String compactName,
-        String... inputFiles) throws IOException {
-      this.vm = vm;
-      this.conf = conf;
-
-      rootTabletDir = new File(tempDir, dirName + "/accumulo/tables/+r/root_tablet");
-      assertTrue(rootTabletDir.mkdirs() || rootTabletDir.isDirectory());
-      oldDatafiles = new HashSet<>();
-      for (String filename : inputFiles) {
-        File file = new File(rootTabletDir, filename);
-        assertTrue(file.createNewFile());
-        oldDatafiles.add(new Path(file.toURI()));
-      }
-
-      this.compactName = compactName;
-
-      File tmpFile = new File(rootTabletDir, compactName + "_tmp");
-      assertTrue(tmpFile.createNewFile());
-      tmpDatafile = new Path(tmpFile.toURI());
-
-      newDatafile = new Path(new File(rootTabletDir, compactName).toURI());
-    }
-
-    void prepareReplacement() throws IOException {
-      prepareReplacement(vm, new Path(rootTabletDir.toURI()), oldDatafiles, compactName);
-    }
-
-    void renameReplacement() throws IOException {
-      renameReplacement(vm, tmpDatafile, newDatafile);
-    }
-
-    public void finishReplacement() throws IOException {
-      finishReplacement(conf, vm, new Path(rootTabletDir.toURI()), oldDatafiles, compactName);
-    }
-
-    public Collection<String> cleanupReplacement(String... expectedFiles) throws IOException {
-      Collection<String> ret =
-          Upgrader9to10.cleanupRootTabletFiles(vm, rootTabletDir.toString()).keySet();
-
-      HashSet<String> expected = new HashSet<>();
-      for (String efile : expectedFiles) {
-        expected.add(new File(rootTabletDir, efile).toURI().toString());
-      }
-
-      assertEquals(expected, new HashSet<>(ret));
-
-      return ret;
-    }
-
-    public void assertFiles(String... files) {
-      HashSet<String> actual = new HashSet<>();
-      File[] children = rootTabletDir.listFiles();
-      if (children != null) {
-        for (File file : children) {
-          actual.add(file.getName());
-        }
-      }
-
-      HashSet<String> expected = new HashSet<>();
-      expected.addAll(Arrays.asList(files));
-
-      assertEquals(expected, actual);
-    }
-  }
-
-  @Test
-  public void testFileReplacement() throws IOException {
-
-    ConfigurationCopy conf = new ConfigurationCopy();
-    conf.set(Property.INSTANCE_VOLUMES, "file:///");
-    conf.set(Property.GENERAL_VOLUME_CHOOSER, RandomVolumeChooser.class.getName());
-
-    try (var vm = VolumeManagerImpl.getLocalForTesting("file:///")) {
-
-      String[] uniqueDirNames = getUniqueNames(4);
-
-      TestWrapper wrapper =
-          new TestWrapper(vm, conf, uniqueDirNames[0], "A00004.rf", "A00002.rf", "F00003.rf");
-      wrapper.prepareReplacement();
-      wrapper.renameReplacement();
-      wrapper.finishReplacement();
-      wrapper.assertFiles("A00004.rf");
-
-      wrapper = new TestWrapper(vm, conf, uniqueDirNames[1], "A00004.rf", "A00002.rf", "F00003.rf");
-      wrapper.prepareReplacement();
-      wrapper.cleanupReplacement("A00002.rf", "F00003.rf");
-      wrapper.assertFiles("A00002.rf", "F00003.rf");
-
-      wrapper = new TestWrapper(vm, conf, uniqueDirNames[2], "A00004.rf", "A00002.rf", "F00003.rf");
-      wrapper.prepareReplacement();
-      wrapper.renameReplacement();
-      wrapper.cleanupReplacement("A00004.rf");
-      wrapper.assertFiles("A00004.rf");
-
-      wrapper = new TestWrapper(vm, conf, uniqueDirNames[3], "A00004.rf", "A00002.rf", "F00003.rf");
-      wrapper.prepareReplacement();
-      wrapper.renameReplacement();
-      wrapper.finishReplacement();
-      wrapper.cleanupReplacement("A00004.rf");
-      wrapper.assertFiles("A00004.rf");
-    }
-  }
-
-  public String[] getUniqueNames(int numOfNames) {
-    String[] result = new String[numOfNames];
-    for (int i = 0; i < result.length; i++) {
-      result[i] = testName() + i;
-    }
-    return result;
-  }
-}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java
new file mode 100644
index 0000000..0b756f0
--- /dev/null
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader10to11Test.java
@@ -0,0 +1,192 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZTABLE_STATE;
+import static org.apache.accumulo.manager.upgrade.Upgrader10to11.buildRepTablePath;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.manager.state.tables.TableState;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.codec.VersionedProperties;
+import org.apache.accumulo.server.conf.store.PropStore;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class Upgrader10to11Test {
+  private static final Logger log = LoggerFactory.getLogger(Upgrader10to11Test.class);
+
+  private InstanceId instanceId = null;
+  private ServerContext context = null;
+  private ZooReaderWriter zrw = null;
+
+  private PropStore propStore = null;
+
+  @BeforeEach
+  public void initMocks() {
+    instanceId = InstanceId.of(UUID.randomUUID());
+    context = createMock(ServerContext.class);
+    zrw = createMock(ZooReaderWriter.class);
+    propStore = createMock(PropStore.class);
+
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
+  }
+
+  @Test
+  void upgradeZooKeeperGoPath() throws Exception {
+
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+    expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once();
+    expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE))
+        .andReturn(TableState.OFFLINE.name().getBytes(UTF_8)).once();
+    zrw.recursiveDelete(buildRepTablePath(instanceId), ZooUtil.NodeMissingPolicy.SKIP);
+    expectLastCall().once();
+
+    expect(propStore.get(TablePropKey.of(instanceId, AccumuloTable.METADATA.tableId())))
+        .andReturn(new VersionedProperties()).once();
+
+    replay(context, zrw, propStore);
+
+    Upgrader10to11 upgrader = new Upgrader10to11();
+    upgrader.upgradeZookeeper(context);
+
+    verify(context, zrw);
+  }
+
+  @Test
+  void upgradeZookeeperNoReplTableNode() throws Exception {
+
+    expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(false).once();
+    replay(context, zrw);
+
+    Upgrader10to11 upgrader = new Upgrader10to11();
+    upgrader.upgradeZookeeper(context);
+
+    verify(context, zrw);
+  }
+
+  @Test
+  void checkReplicationStateOffline() throws Exception {
+
+    expect(context.getPropStore()).andReturn(propStore).anyTimes();
+    expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once();
+    expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE))
+        .andReturn(TableState.OFFLINE.name().getBytes(UTF_8)).once();
+    zrw.recursiveDelete(buildRepTablePath(instanceId), ZooUtil.NodeMissingPolicy.SKIP);
+    expectLastCall().once();
+    expect(propStore.get(TablePropKey.of(instanceId, AccumuloTable.METADATA.tableId())))
+        .andReturn(new VersionedProperties()).once();
+
+    replay(context, zrw, propStore);
+
+    Upgrader10to11 upgrader = new Upgrader10to11();
+
+    upgrader.upgradeZookeeper(context);
+
+    verify(context, zrw);
+  }
+
+  @Test
+  void checkReplicationStateOnline() throws Exception {
+    expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once();
+    expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE))
+        .andReturn(TableState.ONLINE.name().getBytes(UTF_8)).anyTimes();
+    replay(context, zrw);
+
+    Upgrader10to11 upgrader = new Upgrader10to11();
+    assertThrows(IllegalStateException.class, () -> upgrader.upgradeZookeeper(context));
+
+    verify(context, zrw);
+  }
+
+  @Test
+  void checkReplicationStateNoNode() throws Exception {
+    expect(zrw.exists(buildRepTablePath(instanceId))).andReturn(true).once();
+    expect(zrw.getData(buildRepTablePath(instanceId) + ZTABLE_STATE))
+        .andThrow(new KeeperException.NoNodeException("force no node exception")).anyTimes();
+    replay(context, zrw);
+
+    Upgrader10to11 upgrader = new Upgrader10to11();
+    assertThrows(IllegalStateException.class, () -> upgrader.upgradeZookeeper(context));
+
+    verify(context, zrw);
+  }
+
+  @Test
+  public void filterTest() {
+    Map<String,String> entries = new HashMap<>();
+    entries.put("table.file.compress.blocksize", "32K");
+    entries.put("table.file.replication", "5");
+    entries.put("table.group.server", "file,log,srv,future");
+    entries.put("table.iterator.majc.bulkLoadFilter",
+        "20,org.apache.accumulo.server.iterators.MetadataBulkLoadFilter");
+    entries.put("table.iterator.majc.replcombiner",
+        "9,org.apache.accumulo.server.replication.StatusCombiner");
+    entries.put("table.iterator.majc.replcombiner.opt.columns", "stat");
+    entries.put("table.iterator.majc.vers",
+        "10,org.apache.accumulo.core.iterators.user.VersioningIterator");
+    entries.put("table.iterator.majc.vers.opt.maxVersions", "1");
+    entries.put("table.iterator.minc.replcombiner",
+        "9,org.apache.accumulo.server.replication.StatusCombiner");
+    entries.put("table.iterator.minc.replcombiner.opt.columns", "stat");
+    entries.put("table.iterator.minc.vers",
+        "10,org.apache.accumulo.core.iterators.user.VersioningIterator");
+    entries.put("table.iterator.minc.vers.opt.maxVersions", "1");
+    entries.put("table.iterator.scan.replcombiner",
+        "9,org.apache.accumulo.server.replication.StatusCombiner");
+    entries.put("table.iterator.scan.replcombiner.opt.columns", "stat");
+    entries.put("table.iterator.scan.vers",
+        "10,org.apache.accumulo.core.iterators.user.VersioningIterator");
+
+    String REPL_ITERATOR_PATTERN = "^table\\.iterator\\.(majc|minc|scan)\\.replcombiner$";
+    String REPL_COLUMN_PATTERN =
+        "^table\\.iterator\\.(majc|minc|scan)\\.replcombiner\\.opt\\.columns$";
+
+    Pattern p = Pattern.compile("(" + REPL_ITERATOR_PATTERN + "|" + REPL_COLUMN_PATTERN + ")");
+
+    List<String> filtered =
+        entries.keySet().stream().filter(e -> p.matcher(e).find()).collect(Collectors.toList());
+
+    assertEquals(6, filtered.size());
+    log.info("F:{}", filtered);
+  }
+}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java
new file mode 100644
index 0000000..14d19de
--- /dev/null
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java
@@ -0,0 +1,432 @@
+/*
+ * 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.manager.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.manager.upgrade.Upgrader11to12.UPGRADE_FAMILIES;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.mock;
+import static org.easymock.EasyMock.newCapture;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.InstanceId;
+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.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
+import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.data.Stat;
+import org.easymock.Capture;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Upgrader11to12Test {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Upgrader11to12Test.class);
+
+  @Test
+  void upgradeDataFileCF2Test() {
+    String fileName = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/A000000v.rf";
+    Key k = Key.builder().row(new Text("12;")).family(DataFileColumnFamily.NAME)
+        .qualifier(new Text(fileName)).build();
+    Value v = new Value("1234,5678");
+
+    Mutation upgrade = new Mutation(k.getRow());
+    Upgrader11to12.upgradeDataFileCF(k, v, upgrade);
+
+    var pending = upgrade.getUpdates();
+    assertEquals(2, pending.size());
+    // leverage sort order for "expected" values
+    // check file entry converted is in the mutation
+    Iterator<ColumnUpdate> m = pending.iterator();
+    var cu1 = m.next();
+    assertEquals("file", new Text(cu1.getColumnFamily()).toString());
+
+    StoredTabletFile oldFileEntry = StoredTabletFile.of(new Path(fileName));
+    StoredTabletFile updateEnry = StoredTabletFile.of(new String(cu1.getColumnQualifier(), UTF_8));
+
+    assertEquals(oldFileEntry, updateEnry);
+    assertFalse(cu1.isDeleted());
+
+    // check old file entry is deleted is in the mutation
+
+    var cu2 = m.next();
+    assertEquals("file", new Text(cu1.getColumnFamily()).toString());
+    assertEquals(fileName, new String(cu2.getColumnQualifier(), UTF_8));
+    assertTrue(cu2.isDeleted());
+
+  }
+
+  @Test
+  public void processReferencesTest() {
+
+    // create sample data "served" by the mocked scanner
+    TreeMap<Key,Value> scanData = new TreeMap<>();
+    Text row1 = new Text("123");
+
+    String fileName1 = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/A000000v.rf";
+    Key key1 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName1).build();
+    Value value1 = new Value("123,456");
+    scanData.put(key1, value1);
+
+    String fileName2 = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/B000000v.rf";
+    Key key2 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName2).build();
+    Value value2 = new Value("321,654");
+    scanData.put(key2, value2);
+
+    @SuppressWarnings("deprecation")
+    var chopped = ChoppedColumnFamily.NAME;
+    Key chop1 = Key.builder(false).row(row1).family(chopped).qualifier(chopped).build();
+    scanData.put(chop1, null);
+
+    Key extern1 = Key.builder(false).row(row1).family(ExternalCompactionColumnFamily.NAME)
+        .qualifier(ExternalCompactionColumnFamily.NAME).build();
+    scanData.put(extern1, null);
+
+    Text row2 = new Text("234");
+
+    String fileName3 = "hdfs://localhost:8020/accumulo/tables/13/default_tablet/C000000v.rf";
+    Key key3 =
+        Key.builder(false).row(row2).family(DataFileColumnFamily.NAME).qualifier(fileName3).build();
+    Value value3 = new Value("1,2");
+    scanData.put(key3, value3);
+
+    ArrayList<Mutation> mutations = new ArrayList<>();
+
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    upgrader.processReferences(mutations::add, scanData.entrySet(), "accumulo.metadata");
+
+    assertEquals(2, mutations.size());
+
+    var u1 = mutations.get(0);
+    LOG.info("c:{}", u1.prettyPrint());
+    // 2 file add, 2 file delete. 1 chop delete, 1 ext comp delete
+    assertEquals(6, u1.getUpdates().size());
+
+    var u2 = mutations.get(1);
+    LOG.info("c:{}", u2.prettyPrint());
+    // 1 add, 1 delete
+    assertEquals(2, u2.getUpdates().size());
+    assertEquals(1, u2.getUpdates().stream().filter(ColumnUpdate::isDeleted).count());
+
+  }
+
+  @Test
+  public void skipConvertedFileTest() {
+    // create sample data "served" by the mocked scanner
+    TreeMap<Key,Value> scanData = new TreeMap<>();
+    Text row1 = new Text("123");
+
+    // reference already in expected form with fence info.
+    String fileName1 =
+        "{\"path\":\"hdfs://localhost:8020/accumulo/tables/12/default_tablet/A000000v.rf\",\"startRow\":\"\",\"endRow\":\"\"}";
+    Key key1 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName1).build();
+    Value value1 = new Value("123,456");
+    scanData.put(key1, value1);
+
+    String fileName2 = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/B000000v.rf";
+    Key key2 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName2).build();
+    Value value2 = new Value("321,654");
+    scanData.put(key2, value2);
+
+    ArrayList<Mutation> mutations = new ArrayList<>();
+
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    upgrader.processReferences(mutations::add, scanData.entrySet(), "accumulo.metadata");
+
+    assertEquals(1, mutations.size());
+
+    var u1 = mutations.get(0);
+    LOG.info("c:{}", u1.prettyPrint());
+    // 1 add, 1 delete
+    assertEquals(2, u1.getUpdates().size());
+    assertEquals(1, u1.getUpdates().stream().filter(ColumnUpdate::isDeleted).count());
+  }
+
+  @Test
+  void failOnMutationErrorTest() throws Exception {
+
+    BatchWriter batchWriter = mock(BatchWriter.class);
+    Capture<Mutation> capturedUpdate1 = newCapture();
+    batchWriter.addMutation(capture(capturedUpdate1));
+    expectLastCall().andThrow(new MutationsRejectedException(null, List.of(), Map.of(), List.of(),
+        0, new NullPointerException())).once();
+
+    TreeMap<Key,Value> scanData = new TreeMap<>();
+    Text row1 = new Text("123");
+
+    // reference already in expected form with fence info.
+    String fileName1 = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/A000000v.rf";
+    Key key1 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName1).build();
+    Value value1 = new Value("123,456");
+    scanData.put(key1, value1);
+
+    replay(batchWriter);
+    Upgrader11to12 upgrader = new Upgrader11to12();
+
+    assertThrows(IllegalStateException.class, () -> upgrader
+        .processReferences(batchWriter::addMutation, scanData.entrySet(), "accumulo.metadata"));
+
+    verify(batchWriter);
+  }
+
+  @Test
+  void upgradeDataFileCFInvalidPathTest() throws Exception {
+
+    BatchWriter batchWriter = mock(BatchWriter.class);
+    Capture<Mutation> capturedUpdate1 = newCapture();
+    batchWriter.addMutation(capture(capturedUpdate1));
+    // expecting that exception will be called before mutation is updated.
+    expectLastCall().andThrow(new UnsupportedOperationException()).anyTimes();
+
+    // create sample data "served" by the mocked scanner
+    TreeMap<Key,Value> scanData = new TreeMap<>();
+    Text row1 = new Text("123");
+
+    String fileName1 = "bad path";
+    Key key1 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName1).build();
+    Value value1 = new Value("123,456");
+    scanData.put(key1, value1);
+
+    String fileName2 = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/B000000v.rf";
+    Key key2 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName2).build();
+    Value value2 = new Value("321,654");
+    scanData.put(key2, value2);
+
+    replay(batchWriter);
+
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    assertThrows(IllegalArgumentException.class, () -> upgrader
+        .processReferences(batchWriter::addMutation, scanData.entrySet(), "accumulo.metadata"));
+
+    verify(batchWriter);
+  }
+
+  @Test
+  void unexpectedColFailsTest() throws Exception {
+
+    BatchWriter batchWriter = mock(BatchWriter.class);
+    Capture<Mutation> capturedUpdate1 = newCapture();
+    batchWriter.addMutation(capture(capturedUpdate1));
+    // expecting that exception will be called before mutation is updated.
+    expectLastCall().andThrow(new UnsupportedOperationException()).anyTimes();
+
+    // create sample data "served" by the mocked scanner
+    TreeMap<Key,Value> scanData = new TreeMap<>();
+    Text row1 = new Text("123");
+
+    Key key1 = Key.builder(false).row(row1).family(LastLocationColumnFamily.NAME).qualifier("srv1")
+        .build();
+    Value value1 = new Value("123,456");
+    scanData.put(key1, value1);
+
+    replay(batchWriter);
+
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    assertThrows(IllegalStateException.class, () -> upgrader
+        .processReferences(batchWriter::addMutation, scanData.entrySet(), "accumulo.metadata"));
+
+    verify(batchWriter);
+  }
+
+  /**
+   * process 3 rows, 2 should result in no mutations and batch writer addMutation should not be
+   * called for those rows
+   */
+  @Test
+  public void verifyEmptyMutation() {
+    // create sample data "served" by the mocked scanner
+    TreeMap<Key,Value> scanData = new TreeMap<>();
+
+    Text row1 = new Text("1");
+
+    String fileName1 = "hdfs://localhost:8020/accumulo/tables/12/default_tablet/1111000v.rf";
+    Key key1 =
+        Key.builder(false).row(row1).family(DataFileColumnFamily.NAME).qualifier(fileName1).build();
+    Value value1 = new Value("111,222");
+    scanData.put(key1, value1);
+
+    Text row2 = new Text("a");
+
+    // reference already in expected form with fence info.
+    String fileName2 =
+        "{\"path\":\"hdfs://localhost:8020/accumulo/tables/12/default_tablet/A000000v.rf\",\"startRow\":\"\",\"endRow\":\"\"}";
+    Key key2 =
+        Key.builder(false).row(row2).family(DataFileColumnFamily.NAME).qualifier(fileName2).build();
+    Value value2 = new Value("222,333");
+    scanData.put(key2, value2);
+
+    Text row3 = new Text("b");
+
+    // reference already in expected form with fence info.
+    String fileName3 =
+        "{\"path\":\"hdfs://localhost:8020/accumulo/tables/12/default_tablet/BBBB000v.rf\",\"startRow\":\"\",\"endRow\":\"\"}";
+    Key key3 =
+        Key.builder(false).row(row3).family(DataFileColumnFamily.NAME).qualifier(fileName3).build();
+    Value value3 = new Value("333,444");
+    scanData.put(key3, value3);
+
+    ArrayList<Mutation> mutations = new ArrayList<>();
+
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    upgrader.processReferences(mutations::add, scanData.entrySet(), "accumulo.metadata");
+
+    assertEquals(1, mutations.size());
+    var u1 = mutations.get(0);
+    LOG.info("c:{}", u1.prettyPrint());
+    // 1 add, 1 delete
+    assertEquals(2, u1.getUpdates().size());
+    assertEquals(1, u1.getUpdates().stream().filter(ColumnUpdate::isDeleted).count());
+  }
+
+  @Test
+  public void upgradeZooKeeperTest() throws Exception {
+
+    // taken from an uno instance.
+    final byte[] zKRootV1 =
+        "{\"version\":1,\"columnValues\":{\"file\":{\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A0000030.rf\":\"856,15\",\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000r.rf\":\"308,2\"},\"last\":{\"100017f46240004\":\"localhost:9997\"},\"loc\":{\"100017f46240004\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"16\",\"lock\":\"tservers/localhost:9997/zlock#f6a582b9-9583-4553-b179-a7a3852c8332#0000000000$100017f46240004\",\"time\":\"L42\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}\n"
+            .getBytes(UTF_8);
+    final String zKRootV2 =
+        "{\"version\":1,\"columnValues\":{\"file\":{\"{\\\"path\\\":\\\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A0000030.rf\\\",\\\"startRow\\\":\\\"\\\",\\\"endRow\\\":\\\"\\\"}\":\"856,15\",\"{\\\"path\\\":\\\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000r.rf\\\",\\\"startRow\\\":\\\"\\\",\\\"endRow\\\":\\\"\\\"}\":\"308,2\"},\"last\":{\"100017f46240004\":\"localhost:9997\"},\"loc\":{\"100017f46240004\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"16\",\"lock\":\"tservers/localhost:9997/zlock#f6a582b9-9583-4553-b179-a7a3852c8332#0000000000$100017f46240004\",\"time\":\"L42\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}";
+
+    InstanceId iid = InstanceId.of(UUID.randomUUID());
+    Upgrader11to12 upgrader = new Upgrader11to12();
+
+    ServerContext context = createMock(ServerContext.class);
+    ZooReaderWriter zrw = createMock(ZooReaderWriter.class);
+
+    expect(context.getInstanceID()).andReturn(iid).anyTimes();
+    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
+
+    Capture<Stat> statCapture = newCapture();
+    expect(zrw.getData(eq("/accumulo/" + iid.canonical() + "/root_tablet"), capture(statCapture)))
+        .andAnswer(() -> {
+          Stat stat = statCapture.getValue();
+          stat.setCtime(System.currentTimeMillis());
+          stat.setMtime(System.currentTimeMillis());
+          stat.setVersion(123); // default version
+          stat.setDataLength(zKRootV1.length);
+          statCapture.setValue(stat);
+          return zKRootV1;
+        }).once();
+
+    Capture<byte[]> byteCapture = newCapture();
+    expect(zrw.overwritePersistentData(eq("/accumulo/" + iid.canonical() + "/root_tablet"),
+        capture(byteCapture), eq(123))).andReturn(true).once();
+
+    replay(context, zrw);
+
+    upgrader.upgradeZookeeper(context);
+
+    assertEquals(zKRootV2, new String(byteCapture.getValue(), UTF_8));
+
+    verify(context, zrw);
+  }
+
+  @Test
+  public void convertRoot1File() {
+    String root21ZkData =
+        "{\"version\":1,\"columnValues\":{\"file\":{\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A000000v.rf\":\"1368,61\"},\"last\":{\"100025091780006\":\"localhost:9997\"},\"loc\":{\"100025091780006\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"3\",\"lock\":\"tservers/localhost:9997/zlock#9db8961a-4ee9-400e-8e80-3353148baadd#0000000000$100025091780006\",\"time\":\"L53\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}";
+
+    RootTabletMetadata rtm = new RootTabletMetadata(root21ZkData);
+    ArrayList<Mutation> mutations = new ArrayList<>();
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    upgrader.processReferences(mutations::add,
+        rtm.toKeyValues().filter(e -> UPGRADE_FAMILIES.contains(e.getKey().getColumnFamily()))
+            .collect(Collectors.toList()),
+        "accumulo.metadata");
+    assertEquals(1, mutations.size());
+    var mutation = mutations.get(0);
+    rtm.update(mutation);
+
+    LOG.debug("converted column values: {}", rtm.toTabletMetadata().getFiles());
+
+    var files = rtm.toTabletMetadata().getFiles();
+    LOG.info("FILES: {}", rtm.toTabletMetadata().getFilesMap());
+
+    assertEquals(1, files.size());
+    assertTrue(files.contains(StoredTabletFile
+        .of(new Path("hdfs://localhost:8020/accumulo/tables/+r/root_tablet/A000000v.rf"))));
+  }
+
+  @Test
+  public void convertRoot2Files() {
+    String root212ZkData2Files =
+        "{\"version\":1,\"columnValues\":{\"file\":{\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/00000_00000.rf\":\"0,0\",\"hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000c.rf\":\"926,18\"},\"last\":{\"10001a84d7d0005\":\"localhost:9997\"},\"loc\":{\"10001a84d7d0005\":\"localhost:9997\"},\"srv\":{\"dir\":\"root_tablet\",\"flush\":\"2\",\"lock\":\"tservers/localhost:9997/zlock#d21adaa4-0f97-4004-9ff8-cce9dbb6687f#0000000000$10001a84d7d0005\",\"time\":\"L6\"},\"~tab\":{\"~pr\":\"\\u0000\"}}}\n";
+
+    RootTabletMetadata rtm = new RootTabletMetadata(root212ZkData2Files);
+    ArrayList<Mutation> mutations = new ArrayList<>();
+    Upgrader11to12 upgrader = new Upgrader11to12();
+    upgrader.processReferences(mutations::add,
+        rtm.toKeyValues().filter(e -> UPGRADE_FAMILIES.contains(e.getKey().getColumnFamily()))
+            .collect(Collectors.toList()),
+        "accumulo.metadata");
+    assertEquals(1, mutations.size());
+    var mutation = mutations.get(0);
+    rtm.update(mutation);
+
+    LOG.debug("converted column values: {}", rtm.toTabletMetadata());
+
+    var files = rtm.toTabletMetadata().getFiles();
+    LOG.info("FILES: {}", rtm.toTabletMetadata().getFilesMap());
+
+    assertEquals(2, files.size());
+    assertTrue(files.contains(StoredTabletFile
+        .of(new Path("hdfs://localhost:8020/accumulo/tables/+r/root_tablet/00000_00000.rf"))));
+    assertTrue(files.contains(StoredTabletFile
+        .of(new Path("hdfs://localhost:8020/accumulo/tables/+r/root_tablet/F000000c.rf"))));
+  }
+
+}
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader9to10Test.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader9to10Test.java
deleted file mode 100644
index 290208b..0000000
--- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader9to10Test.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.
- */
-package org.apache.accumulo.manager.upgrade;
-
-import static org.apache.accumulo.core.Constants.BULK_PREFIX;
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ColumnUpdate;
-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.accumulo.core.gc.ReferenceFile;
-import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.volume.Volume;
-import org.apache.accumulo.core.volume.VolumeImpl;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.gc.AllVolumesDirectory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class Upgrader9to10Test {
-  private static Logger log = LoggerFactory.getLogger(Upgrader9to10Test.class);
-
-  private static final String VOL_PROP = "hdfs://nn1:8020/accumulo";
-  private static final TableId tableId5a = TableId.of("5a");
-
-  @Test
-  public void testSwitchRelativeDeletes() {
-    Path resolved = Upgrader9to10.resolveRelativeDelete("/5a/t-0005", VOL_PROP);
-    assertEquals(new Path(VOL_PROP + "/tables/5a/t-0005"), resolved);
-    var allVolumesDir = new AllVolumesDirectory(tableId5a, "t-0005");
-    var ref1 = Upgrader9to10.switchToAllVolumes(resolved);
-    compareReferences(allVolumesDir, ref1);
-
-    resolved = Upgrader9to10.resolveRelativeDelete("/5a/" + BULK_PREFIX + "0005", VOL_PROP);
-    assertEquals(new Path(VOL_PROP + "/tables/5a/" + BULK_PREFIX + "0005"), resolved);
-    ref1 = ReferenceFile.forFile(tableId5a, VOL_PROP + "/tables/5a/" + BULK_PREFIX + "0005");
-    var ref2 = Upgrader9to10.switchToAllVolumes(resolved);
-    compareReferences(ref1, ref2);
-
-    resolved = Upgrader9to10.resolveRelativeDelete("/5a/t-0005/F0009.rf", VOL_PROP);
-    assertEquals(new Path(VOL_PROP + "/tables/5a/t-0005/F0009.rf"), resolved);
-    ref1 = ReferenceFile.forFile(tableId5a, VOL_PROP + "/tables/5a/t-0005/F0009.rf");
-    ref2 = Upgrader9to10.switchToAllVolumes(resolved);
-    compareReferences(ref1, ref2);
-  }
-
-  private void compareReferences(ReferenceFile ref1, ReferenceFile ref2) {
-    assertEquals(ref1.getMetadataEntry(), ref2.getMetadataEntry());
-    assertEquals(ref1.tableId, ref2.tableId);
-  }
-
-  @Test
-  public void testBadRelativeDeleteTooShort() {
-    assertThrows(IllegalStateException.class,
-        () -> Upgrader9to10.resolveRelativeDelete("/5a", VOL_PROP));
-  }
-
-  @Test
-  public void testBadRelativeDeleteTooLong() throws Exception {
-    assertThrows(IllegalStateException.class,
-        () -> Upgrader9to10.resolveRelativeDelete("/5a/5a/t-0005/F0009.rf", VOL_PROP));
-  }
-
-  @Test
-  public void testSwitchAllVolumes() {
-    Path resolved = Upgrader9to10
-        .resolveRelativeDelete("hdfs://localhost:9000/accumulo/tables/5a/t-0005", VOL_PROP);
-    var allVolumesDir = new AllVolumesDirectory(tableId5a, "t-0005");
-    var ref1 = Upgrader9to10.switchToAllVolumes(resolved);
-    compareReferences(allVolumesDir, ref1);
-
-    resolved = Upgrader9to10.resolveRelativeDelete(
-        "hdfs://localhost:9000/accumulo/tables/5a/" + BULK_PREFIX + "0005", VOL_PROP);
-    ref1 = ReferenceFile.forFile(tableId5a,
-        "hdfs://localhost:9000/accumulo/tables/5a/" + BULK_PREFIX + "0005");
-    var ref2 = Upgrader9to10.switchToAllVolumes(resolved);
-    compareReferences(ref1, ref2);
-
-    resolved = Upgrader9to10.resolveRelativeDelete(
-        "hdfs://localhost:9000/accumulo/tables/5a/t-0005/C0009.rf", VOL_PROP);
-    ref1 = ReferenceFile.forFile(tableId5a,
-        "hdfs://localhost:9000/accumulo/tables/5a/t-0005/C0009.rf");
-    ref2 = Upgrader9to10.switchToAllVolumes(resolved);
-    compareReferences(ref1, ref2);
-  }
-
-  @Test
-  public void testUpgradeDir() {
-    assertEquals("t-0005",
-        Upgrader9to10.upgradeDirColumn("hdfs://localhost:9000/accumulo/tables/5a/t-0005"));
-    assertEquals("t-0005", Upgrader9to10.upgradeDirColumn("../5a/t-0005"));
-    assertEquals("t-0005", Upgrader9to10.upgradeDirColumn("/t-0005"));
-    assertEquals("t-0005", Upgrader9to10.upgradeDirColumn("t-0005"));
-  }
-
-  String tableName = Ample.DataLevel.USER.metaTable();
-  String volumeUpgrade = "file:///accumulo";
-
-  // mock objects for testing relative path replacement
-  private void setupMocks(AccumuloClient c, VolumeManager fs, SortedMap<Key,Value> map,
-      List<Mutation> results) throws Exception {
-    Scanner scanner = createMock(Scanner.class);
-    // buffer all the mutations that are created so we can verify they are correct
-    BatchWriter writer = new BatchWriter() {
-      List<Mutation> buffer = new ArrayList<>();
-
-      @Override
-      public void addMutation(Mutation m) throws MutationsRejectedException {
-        buffer.add(m);
-      }
-
-      @Override
-      public void addMutations(Iterable<Mutation> iterable) throws MutationsRejectedException {
-        iterable.forEach(buffer::add);
-      }
-
-      @Override
-      public void flush() throws MutationsRejectedException {}
-
-      @Override
-      // simulate the close by adding all to results and preventing anymore adds
-      public void close() throws MutationsRejectedException {
-        results.addAll(buffer);
-        buffer = null;
-      }
-    };
-
-    expect(c.createScanner(tableName, Authorizations.EMPTY)).andReturn(scanner).anyTimes();
-    expect(c.createBatchWriter(tableName)).andReturn(writer).anyTimes();
-    expect(scanner.iterator()).andReturn(map.entrySet().iterator()).anyTimes();
-
-    // void methods
-    scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-    expectLastCall().anyTimes();
-    scanner.close();
-    expectLastCall().anyTimes();
-
-    replay(c, fs, scanner);
-  }
-
-  @Test
-  public void noRelativePaths() throws Exception {
-    VolumeManager fs = createMock(VolumeManager.class);
-    AccumuloClient c = createMock(AccumuloClient.class);
-    expect(fs.exists(anyObject())).andReturn(true).anyTimes();
-
-    SortedMap<Key,Value> map = new TreeMap<>();
-    map.put(new Key("1<", "file", "hdfs://nn1:8020/accumulo/tables/1/default_tablet/A000001c.rf"),
-        new Value());
-    map.put(new Key("1<", "file", "hdfs://nn1:8020/accumulo/tables/1/default_tablet/F000001m.rf"),
-        new Value());
-    map.put(new Key("1<", "file", "file://nn1:8020/accumulo/tables/1/t-0005/F000004x.rf"),
-        new Value());
-    map.put(new Key("1<", "file", "file://volume23:8000/accumulo/tables/1/t-1234/F0000054.rf"),
-        new Value());
-
-    List<Mutation> results = new ArrayList<>();
-
-    setupMocks(c, fs, map, results);
-    assertFalse(Upgrader9to10.checkForRelativePaths(c, fs, tableName, volumeUpgrade),
-        "Invalid Relative path check");
-    assertTrue(results.isEmpty());
-  }
-
-  @Test
-  public void filesDontExistAfterReplacingRelatives() throws Exception {
-    AccumuloClient c = createMock(AccumuloClient.class);
-    VolumeManager fs = createMock(VolumeManager.class);
-    SortedMap<Key,Value> map = new TreeMap<>();
-    map.put(new Key("1b;row_000050", "file", "../1b/default_tablet/A000001c.rf"), new Value("1"));
-    map.put(new Key("1b;row_000050", "file", "../1b/default_tablet/F000001m.rf"), new Value("2"));
-
-    expect(fs.exists(anyObject(Path.class))).andReturn(false).anyTimes();
-
-    setupMocks(c, fs, map, new ArrayList<>());
-    assertThrows(IllegalArgumentException.class,
-        () -> Upgrader9to10.checkForRelativePaths(c, fs, tableName, volumeUpgrade));
-  }
-
-  @Test
-  public void missingUpgradeRelativeProperty() throws Exception {
-    AccumuloClient c = createMock(AccumuloClient.class);
-    VolumeManager fs = createMock(VolumeManager.class);
-    SortedMap<Key,Value> map = new TreeMap<>();
-    map.put(new Key("1b;row_000050", "file", "../1b/default_tablet/A000001c.rf"), new Value("1"));
-    map.put(new Key("1b;row_000050", "file", "../1b/default_tablet/F000001m.rf"), new Value("2"));
-
-    expect(fs.exists(anyObject(Path.class))).andReturn(false).anyTimes();
-
-    setupMocks(c, fs, map, new ArrayList<>());
-    assertThrows(IllegalArgumentException.class,
-        () -> Upgrader9to10.checkForRelativePaths(c, fs, tableName, ""));
-  }
-
-  @Test
-  public void replaceRelatives() throws Exception {
-    AccumuloClient c = createMock(AccumuloClient.class);
-    VolumeManager fs = createMock(VolumeManager.class);
-    expect(fs.exists(anyObject())).andReturn(true).anyTimes();
-
-    SortedMap<Key,Value> map = new TreeMap<>();
-    map.put(new Key("1b;row_000050", "file", "../1b/default_tablet/A000001c.rf"), new Value("1"));
-    map.put(new Key("1b;row_000050", "file", "../1b/default_tablet/F000001m.rf"), new Value("2"));
-    map.put(new Key("1b;row_000050", "file", "../1b/t-000008t/F000004x.rf"), new Value("3"));
-    map.put(new Key("1b;row_000050", "file", "/t-000008t/F0000054.rf"), new Value("4"));
-    map.put(new Key("1b<", "file", "../1b/default_tablet/A000001c.rf"), new Value("1"));
-    map.put(new Key("1b<", "file", "../1b/default_tablet/F000001m.rf"), new Value("2"));
-    map.put(new Key("1b<", "file", "../1b/t-000008t/F000004x.rf"), new Value("3"));
-    map.put(new Key("1b<", "file", "/t-000008t/F0000054.rf"), new Value("4"));
-    map.put(new Key("1b<", "file", "hdfs://nn1:8020/accumulo/tables/1b/t-000008t/A0000098.rf"),
-        new Value("5"));
-    map.put(new Key("1b<", "file", "hdfs://nn1:8020/accumulo/tables/1b/t-000008t/F0000098.rf"),
-        new Value("5"));
-
-    List<Mutation> expected = new ArrayList<>();
-    expected.add(replaceMut("1b;row_000050", "file:/accumulo/tables/1b/default_tablet/A000001c.rf",
-        "1", "../1b/default_tablet/A000001c.rf"));
-    expected.add(replaceMut("1b;row_000050", "file:/accumulo/tables/1b/default_tablet/F000001m.rf",
-        "2", "../1b/default_tablet/F000001m.rf"));
-    expected.add(replaceMut("1b;row_000050", "file:/accumulo/tables/1b/t-000008t/F000004x.rf", "3",
-        "../1b/t-000008t/F000004x.rf"));
-    expected.add(replaceMut("1b;row_000050", "file:/accumulo/tables/1b/t-000008t/F0000054.rf", "4",
-        "/t-000008t/F0000054.rf"));
-    expected.add(replaceMut("1b<", "file:/accumulo/tables/1b/default_tablet/A000001c.rf", "1",
-        "../1b/default_tablet/A000001c.rf"));
-    expected.add(replaceMut("1b<", "file:/accumulo/tables/1b/default_tablet/F000001m.rf", "2",
-        "../1b/default_tablet/F000001m.rf"));
-    expected.add(replaceMut("1b<", "file:/accumulo/tables/1b/t-000008t/F000004x.rf", "3",
-        "../1b/t-000008t/F000004x.rf"));
-    expected.add(replaceMut("1b<", "file:/accumulo/tables/1b/t-000008t/F0000054.rf", "4",
-        "/t-000008t/F0000054.rf"));
-
-    List<Mutation> results = new ArrayList<>();
-
-    setupMocks(c, fs, map, results);
-    Upgrader9to10.replaceRelativePaths(c, fs, tableName, volumeUpgrade);
-    verifyPathsReplaced(expected, results);
-  }
-
-  @Test
-  public void normalizeVolume() throws Exception {
-    String uglyVolume = "hdfs://nn.somewhere.com:86753/accumulo/blah/.././/bad/bad2/../.././/////";
-
-    AccumuloClient c = createMock(AccumuloClient.class);
-    VolumeManager fs = createMock(VolumeManager.class);
-    expect(fs.exists(anyObject())).andReturn(true).anyTimes();
-    SortedMap<Key,Value> map = new TreeMap<>();
-    map.put(new Key("1b<", "file", "../1b/t-000008t/F000004x.rf"), new Value("1"));
-    map.put(new Key("1b<", "file", "/t-000008t/F0000054.rf"), new Value("2"));
-    List<Mutation> results = new ArrayList<>();
-    List<Mutation> expected = new ArrayList<>();
-    expected.add(
-        replaceMut("1b<", "hdfs://nn.somewhere.com:86753/accumulo/tables/1b/t-000008t/F000004x.rf",
-            "1", "../1b/t-000008t/F000004x.rf"));
-    expected.add(
-        replaceMut("1b<", "hdfs://nn.somewhere.com:86753/accumulo/tables/1b/t-000008t/F0000054.rf",
-            "2", "/t-000008t/F0000054.rf"));
-
-    setupMocks(c, fs, map, results);
-    Upgrader9to10.replaceRelativePaths(c, fs, tableName, uglyVolume);
-    verifyPathsReplaced(expected, results);
-  }
-
-  private Mutation replaceMut(String row, String cq, String val, String delete) {
-    Mutation m = new Mutation(row);
-    m.at().family("file").qualifier(cq).put(new Value(val));
-    m.at().family("file").qualifier(delete).delete();
-    return m;
-  }
-
-  /**
-   * Make sure mutations are all the same, in the correct order
-   */
-  private void verifyPathsReplaced(List<Mutation> expected, List<Mutation> results) {
-    Iterator<Mutation> expectIter = expected.iterator();
-    int deleteCount = 0;
-    int updateCount = 0;
-    for (Mutation mut : results) {
-      Mutation next = expectIter.next();
-      Iterator<ColumnUpdate> nextUpdates = next.getUpdates().iterator();
-      assertEquals(next.getUpdates().size(), mut.getUpdates().size());
-      assertEquals(new Text(next.getRow()), new Text(mut.getRow()));
-
-      // check updates are all the same
-      for (ColumnUpdate update : mut.getUpdates()) {
-        ColumnUpdate nextUpdate = nextUpdates.next();
-        Text cq = new Text(nextUpdate.getColumnQualifier());
-        log.debug("Checking for expected columnUpdate: " + cq + " deleted? " + update.isDeleted());
-        assertEquals(cq, new Text(update.getColumnQualifier()));
-        if (update.isDeleted()) {
-          deleteCount++;
-        } else {
-          updateCount++;
-          assertEquals(new Text(nextUpdate.getValue()), new Text(update.getValue()));
-        }
-      }
-    }
-
-    assertEquals(deleteCount, updateCount, "Replacements should have update for every delete");
-  }
-
-  @Test
-  public void testDropSortedMapWALs() throws IOException {
-    Configuration hadoopConf = new Configuration();
-    ConfigurationCopy conf = new ConfigurationCopy();
-    FileSystem fs = new Path("file:///").getFileSystem(hadoopConf);
-
-    List<String> volumes = Arrays.asList("/vol1/", "/vol2/");
-    Collection<Volume> vols =
-        volumes.stream().map(s -> new VolumeImpl(fs, s)).collect(Collectors.toList());
-    Set<String> fullyQualifiedVols = Set.of("file://vol1/", "file://vol2/");
-    Set<String> recoveryDirs =
-        Set.of("file://vol1/accumulo/recovery", "file://vol2/accumulo/recovery");
-    conf.set(Property.INSTANCE_VOLUMES, String.join(",", fullyQualifiedVols));
-
-    ServerContext context = createMock(ServerContext.class);
-    Path recoveryDir1 = new Path("file://vol1/accumulo/recovery");
-    Path recoveryDir2 = new Path("file://vol2/accumulo/recovery");
-    VolumeManager volumeManager = createMock(VolumeManager.class);
-
-    FileStatus[] dirs = new FileStatus[2];
-    dirs[0] = createMock(FileStatus.class);
-    Path dir0 = new Path("file://vol1/accumulo/recovery/A123456789");
-    FileStatus[] dir0Files = new FileStatus[1];
-    dir0Files[0] = createMock(FileStatus.class);
-    dirs[1] = createMock(FileStatus.class);
-    Path dir1 = new Path("file://vol1/accumulo/recovery/B123456789");
-    FileStatus[] dir1Files = new FileStatus[1];
-    dir1Files[0] = createMock(FileStatus.class);
-    Path part1Dir = new Path("file://vol1/accumulo/recovery/B123456789/part-r-0000");
-
-    expect(context.getVolumeManager()).andReturn(volumeManager).once();
-    expect(context.getConfiguration()).andReturn(conf).once();
-    expect(context.getHadoopConf()).andReturn(hadoopConf).once();
-    expect(context.getRecoveryDirs()).andReturn(recoveryDirs).once();
-    expect(volumeManager.getVolumes()).andReturn(vols).once();
-
-    expect(volumeManager.exists(recoveryDir1)).andReturn(true).once();
-    expect(volumeManager.exists(recoveryDir2)).andReturn(false).once();
-    expect(volumeManager.listStatus(recoveryDir1)).andReturn(dirs).once();
-    expect(dirs[0].getPath()).andReturn(dir0).once();
-    expect(volumeManager.listStatus(dir0)).andReturn(dir0Files).once();
-    expect(dir0Files[0].isDirectory()).andReturn(false).once();
-
-    expect(dirs[1].getPath()).andReturn(dir1).once();
-    expect(volumeManager.listStatus(dir1)).andReturn(dir1Files).once();
-    expect(dir1Files[0].isDirectory()).andReturn(true).once();
-    expect(dir1Files[0].getPath()).andReturn(part1Dir).once();
-    expect(volumeManager.deleteRecursively(dir1)).andReturn(true).once();
-
-    replay(context, volumeManager, dirs[0], dirs[1], dir0Files[0], dir1Files[0]);
-    Upgrader9to10.dropSortedMapWALFiles(context);
-  }
-}
diff --git a/server/manager/src/test/resources/conf/accumulo-site.xml b/server/manager/src/test/resources/conf/accumulo-site.xml
deleted file mode 100644
index fedf3c9..0000000
--- a/server/manager/src/test/resources/conf/accumulo-site.xml
+++ /dev/null
@@ -1,104 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
-<configuration>
-	<!--
-  Put your site-specific accumulo configurations here.
-
-  The available configuration values along with their defaults
-  are documented in docs/config.html
-
-  Unless you are simply testing at your workstation, you will most 
-  definitely need to change the three entries below.
-	-->
-
-  <property>
-    <name>instance.zookeeper.host</name>
-    <value>localhost:2181</value>
-    <description>comma separated list of zookeeper servers</description>
-  </property>
-
-  <property>
-    <name>instance.secret</name>
-    <value>DEFAULT</value>
-    <description>A secret unique to a given instance that all servers must know in order to
-      communicate with one another. Change it before initialization. To change it later use
-      ./bin/accumulo org.apache.accumulo.server.util.ChangeSecret, and then update this file.
-    </description>
-  </property>
-
-  <property>
-    <name>tserver.memory.maps.max</name>
-    <value>80M</value>
-  </property>
-
-  <property>
-    <name>tserver.cache.data.size</name>
-    <value>7M</value>
-  </property>
-
-  <property>
-    <name>tserver.cache.index.size</name>
-    <value>20M</value>
-  </property>
-
-  <property>
-    <name>tserver.sort.buffer.size</name>
-    <value>50M</value>
-  </property>
-
-  <property>
-    <name>tserver.wal.max.size</name>
-    <value>100M</value>
-  </property>
-
-  <property>
-    <name>general.classpaths</name>
-    <!--
-       Add the following for hadoop-2.0
-       $HADOOP_PREFIX/share/hadoop/common/.*.jar,
-       $HADOOP_PREFIX/share/hadoop/common/lib/.*.jar,
-       $HADOOP_PREFIX/share/hadoop/hdfs/.*.jar,
-       $HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar,
-       $HADOOP_PREFIX/share/hadoop/yarn/.*.jar,
-    -->
-    <value>
-      $ACCUMULO_HOME/server/target/classes/,
-      $ACCUMULO_HOME/lib/accumulo-server.jar,
-      $ACCUMULO_HOME/core/target/classes/,
-      $ACCUMULO_HOME/lib/accumulo-core.jar,
-      $ACCUMULO_HOME/start/target/classes/,
-      $ACCUMULO_HOME/lib/accumulo-start.jar,
-      $ACCUMULO_HOME/proxy/target/classes/,
-      $ACCUMULO_HOME/lib/accumulo-proxy.jar,
-      $ACCUMULO_HOME/lib/[^.].*.jar,
-      $ZOOKEEPER_HOME/zookeeper[^.].*.jar,
-      $HADOOP_CONF_DIR,
-      $HADOOP_PREFIX/[^.].*.jar,
-      $HADOOP_PREFIX/lib/[^.].*.jar,
-    </value>
-    <description>Classpaths that accumulo checks for updates and class files.
-      When using the Security Manager, please remove the ".../target/classes/" values.
-    </description>
-  </property>
-</configuration>
diff --git a/server/master/.gitignore b/server/master/.gitignore
deleted file mode 100644
index 55d7f58..0000000
--- a/server/master/.gitignore
+++ /dev/null
@@ -1,36 +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.
-#
-
-# Maven ignores
-/target/
-
-# IDE ignores
-/.settings/
-/.project
-/.classpath
-/.pydevproject
-/.idea
-/*.iml
-/*.ipr
-/*.iws
-/nbproject/
-/nbactions.xml
-/nb-configuration.xml
-/.vscode/
-/.factorypath
diff --git a/server/master/pom.xml b/server/master/pom.xml
deleted file mode 100644
index 3987fad..0000000
--- a/server/master/pom.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.accumulo</groupId>
-    <artifactId>accumulo-project</artifactId>
-    <version>2.1.3-SNAPSHOT</version>
-    <relativePath>../../pom.xml</relativePath>
-  </parent>
-  <!-- should be removed in 3.0 -->
-  <artifactId>accumulo-master</artifactId>
-  <packaging>pom</packaging>
-  <name>Apache Accumulo Master Server (Deprecated)</name>
-  <description>The master server has been replaced by manager.</description>
-  <distributionManagement>
-    <relocation>
-      <artifactId>accumulo-manager</artifactId>
-    </relocation>
-  </distributionManagement>
-</project>
diff --git a/server/monitor/pom.xml b/server/monitor/pom.xml
index f681592..bb4c126 100644
--- a/server/monitor/pom.xml
+++ b/server/monitor/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-monitor</artifactId>
@@ -45,6 +45,10 @@
       <artifactId>gson</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>jakarta.inject</groupId>
       <artifactId>jakarta.inject-api</artifactId>
     </dependency>
diff --git a/server/monitor/src/main/appended-resources/META-INF/LICENSE b/server/monitor/src/main/appended-resources/META-INF/LICENSE
index e4628b6..021fe73 100644
--- a/server/monitor/src/main/appended-resources/META-INF/LICENSE
+++ b/server/monitor/src/main/appended-resources/META-INF/LICENSE
@@ -53,7 +53,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/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 39fe11d..10ccf19 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -18,9 +18,9 @@
  */
 package org.apache.accumulo.monitor;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.HOURS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.net.InetAddress;
 import java.net.URL;
@@ -46,35 +46,35 @@
 import jakarta.inject.Singleton;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
 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.gc.thrift.GCMonitorService;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
-import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
-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.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.monitor.rest.compactions.external.ExternalCompactionInfo;
@@ -82,7 +82,6 @@
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.server.util.TableInfoUtil;
@@ -101,6 +100,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Serve manager statistics with an embedded web server.
  */
@@ -112,12 +113,12 @@
   private final long START_TIME;
 
   public static void main(String[] args) throws Exception {
-    try (Monitor monitor = new Monitor(new ServerOpts(), args)) {
+    try (Monitor monitor = new Monitor(new ConfigOpts(), args)) {
       monitor.runServer();
     }
   }
 
-  Monitor(ServerOpts opts, String[] args) {
+  Monitor(ConfigOpts opts, String[] args) {
     super("monitor", opts, args);
     START_TIME = System.currentTimeMillis();
   }
@@ -428,8 +429,12 @@
       var path = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZGC_LOCK);
       List<String> locks = ServiceLock.validateAndSort(path, zk.getChildren(path.toString()));
       if (locks != null && !locks.isEmpty()) {
-        address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0)), UTF_8))
-            .getAddress(Service.GC_CLIENT);
+        address = ServiceLockData.parse(zk.getData(path + "/" + locks.get(0)))
+            .map(sld -> sld.getAddress(ThriftService.GC)).orElse(null);
+        if (address == null) {
+          log.warn("Unable to contact the garbage collector (no address)");
+          return null;
+        }
         GCMonitorService.Client client =
             ThriftUtil.getClient(ThriftClientTypes.GC, address, context);
         try {
@@ -486,14 +491,10 @@
     }
     HostAndPort monitorHostAndPort = HostAndPort.fromParts(advertiseHost, livePort);
     log.debug("Using {} to advertise monitor location in ZooKeeper", monitorHostAndPort);
-    try {
-      monitorLock.replaceLockData(monitorHostAndPort.toString().getBytes(UTF_8));
-    } catch (KeeperException | InterruptedException e) {
-      throw new IllegalStateException("Exception updating monitor lock with host and port", e);
-    }
 
     MetricsInfo metricsInfo = getContext().getMetricsInfo();
     metricsInfo.addServiceTags(getApplicationName(), monitorHostAndPort);
+    metricsInfo.addMetricsProducers(this);
     metricsInfo.init();
 
     try {
@@ -655,7 +656,7 @@
     }
     if (System.nanoTime() - ecInfoFetchedNanos > fetchTimeNanos) {
       log.info("User initiated fetch of External Compaction info");
-      Map<String,List<HostAndPort>> compactors =
+      Map<String,Set<HostAndPort>> compactors =
           ExternalCompactionUtil.getCompactorAddrs(getContext());
       log.debug("Found compactors: " + compactors);
       ecInfo.setFetchedTimeMillis(System.currentTimeMillis());
@@ -832,7 +833,8 @@
     while (true) {
       MoniterLockWatcher monitorLockWatcher = new MoniterLockWatcher();
       monitorLock = new ServiceLock(zoo.getZooKeeper(), monitorLockPath, zooLockUUID);
-      monitorLock.lock(monitorLockWatcher, new byte[0]);
+      monitorLock.lock(monitorLockWatcher,
+          new ServiceLockData(zooLockUUID, getHostname(), ThriftService.NONE));
 
       monitorLockWatcher.waitForChange();
 
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java
index 634bc00..33ab365 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/XMLResource.java
@@ -27,7 +27,7 @@
 import jakarta.ws.rs.core.Response.Status;
 
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.monitor.rest.manager.ManagerResource;
 import org.apache.accumulo.monitor.rest.tables.TablesResource;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportInformation.java
index 152cde1..39ed451 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportInformation.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.monitor.rest.bulkImports;
 
-import org.apache.accumulo.core.master.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
 
 /**
  * Stores bulk import in a JSON object
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java
index 2dde90e..40ae6f2 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/BulkImportResource.java
@@ -26,9 +26,9 @@
 import jakarta.ws.rs.Produces;
 import jakarta.ws.rs.core.MediaType;
 
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 
 /**
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/TabletServerBulkImportInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/TabletServerBulkImportInformation.java
index 0342549..168065d 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/TabletServerBulkImportInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/bulkImports/TabletServerBulkImportInformation.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.monitor.rest.bulkImports;
 
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 
 /**
  * Stores tserver bulk import information
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java
index 3f9a8c9..5a71fa6 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionInfo.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.monitor.rest.compactions;
 
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 
 /**
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java
index 236c6f8..8ceff7f 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/CompactionsResource.java
@@ -27,10 +27,11 @@
 import jakarta.ws.rs.core.MediaType;
 
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Generate a new Compaction list JSON object
  *
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java
index ebd133b..8724f75 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/CoordinatorInfo.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.monitor.rest.compactions.external;
 
-import java.util.List;
 import java.util.Optional;
+import java.util.Set;
 
-import org.apache.accumulo.core.util.HostAndPort;
+import com.google.common.net.HostAndPort;
 
 public class CoordinatorInfo {
 
@@ -35,7 +35,7 @@
     server = serverOpt.map(HostAndPort::toString).orElse("none");
     var queueToCompactors = ecInfo.getCompactors();
     numQueues = queueToCompactors.size();
-    numCompactors = queueToCompactors.values().stream().mapToInt(List::size).sum();
+    numCompactors = queueToCompactors.values().stream().mapToInt(Set::size).sum();
     lastContact = System.currentTimeMillis() - ecInfo.getFetchedTimeMillis();
   }
 }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java
index b4639e3..251eb16 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/compactions/external/ExternalCompactionInfo.java
@@ -19,11 +19,11 @@
 package org.apache.accumulo.monitor.rest.compactions.external;
 
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
 
-import org.apache.accumulo.core.util.HostAndPort;
+import com.google.common.net.HostAndPort;
 
 /**
  * Bag of everything going on with external compactions.
@@ -31,7 +31,7 @@
 public class ExternalCompactionInfo {
 
   private Optional<HostAndPort> coordinatorHost;
-  private Map<String,List<HostAndPort>> compactors = new HashMap<>();
+  private Map<String,Set<HostAndPort>> compactors = new HashMap<>();
   private long fetchedTimeMillis;
 
   public void setCoordinatorHost(Optional<HostAndPort> coordinatorHost) {
@@ -42,11 +42,11 @@
     return coordinatorHost;
   }
 
-  public Map<String,List<HostAndPort>> getCompactors() {
+  public Map<String,Set<HostAndPort>> getCompactors() {
     return compactors;
   }
 
-  public void setCompactors(Map<String,List<HostAndPort>> compactors) {
+  public void setCompactors(Map<String,Set<HostAndPort>> compactors) {
     this.compactors = compactors;
   }
 
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java
index 9d491cd..c2c20cc 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java
@@ -33,7 +33,7 @@
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.manager.thrift.DeadServer;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.monitor.rest.logs.DeadLoggerInformation;
@@ -102,7 +102,7 @@
       List<String> managers = monitor.getContext().getManagerLocations();
 
       String manager =
-          managers.isEmpty() ? "Down" : AddressUtil.parseAddress(managers.get(0), false).getHost();
+          managers.isEmpty() ? "Down" : AddressUtil.parseAddress(managers.get(0)).getHost();
       int onlineTabletServers = mmi.tServerInfo.size();
       int totalTabletServers = tservers.size();
       int tablets = monitor.getTotalTabletCount();
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationInformation.java
deleted file mode 100644
index bf3f510..0000000
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationInformation.java
+++ /dev/null
@@ -1,56 +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.monitor.rest.replication;
-
-/**
- * Generates the replication information as a JSON object
- *
- * @since 2.0.0
- */
-@Deprecated
-public class ReplicationInformation {
-
-  // Variable names become JSON keys
-  public String tableName;
-  public String peerName;
-  public String remoteIdentifier;
-  public String replicaSystemType;
-
-  public long filesNeedingReplication;
-
-  public ReplicationInformation() {}
-
-  /**
-   * Stores new replication information
-   *
-   * @param tableName Name of the table being replicated
-   * @param peerName Name of the peer
-   * @param remoteIdentifier Identifier of the remote
-   * @param replicaSystemType System type replica
-   * @param filesNeedingReplication Number of files needing replication
-   */
-  public ReplicationInformation(String tableName, String peerName, String remoteIdentifier,
-      String replicaSystemType, long filesNeedingReplication) {
-    this.tableName = tableName;
-    this.peerName = peerName;
-    this.remoteIdentifier = remoteIdentifier;
-    this.replicaSystemType = replicaSystemType;
-    this.filesNeedingReplication = filesNeedingReplication;
-  }
-}
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java
deleted file mode 100644
index fc8f636..0000000
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/replication/ReplicationResource.java
+++ /dev/null
@@ -1,222 +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.monitor.rest.replication;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import jakarta.inject.Inject;
-import jakarta.ws.rs.GET;
-import jakarta.ws.rs.Path;
-import jakarta.ws.rs.Produces;
-import jakarta.ws.rs.core.MediaType;
-
-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.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.conf.Property;
-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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.monitor.Monitor;
-import org.apache.accumulo.server.replication.ReplicaSystem;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Generates the replication table with information from the Monitor
- *
- * @since 2.0.0
- */
-@SuppressWarnings("deprecation")
-@Path("/replication")
-@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-@Deprecated
-public class ReplicationResource {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationResource.class);
-
-  @Inject
-  private Monitor monitor;
-
-  /**
-   * Generates the replication table as a JSON object
-   *
-   * @return Replication list
-   */
-  @GET
-  public List<ReplicationInformation> getReplicationInformation()
-      throws AccumuloException, AccumuloSecurityException {
-    final AccumuloClient client = monitor.getContext();
-
-    final TableOperations tops = client.tableOperations();
-
-    final Map<String,String> properties = client.instanceOperations().getSystemConfiguration();
-    final Map<String,String> peers = new HashMap<>();
-    final String definedPeersPrefix = Property.REPLICATION_PEERS.getKey();
-    final ReplicaSystemFactory replicaSystemFactory = new ReplicaSystemFactory();
-
-    // Get the defined peers and what ReplicaSystem impl they're using
-    for (Entry<String,String> property : properties.entrySet()) {
-      String key = property.getKey();
-      // Filter out cruft that we don't want
-      if (key.startsWith(definedPeersPrefix)
-          && !key.startsWith(Property.REPLICATION_PEER_USER.getKey())
-          && !key.startsWith(Property.REPLICATION_PEER_PASSWORD.getKey())) {
-        String peerName = property.getKey().substring(definedPeersPrefix.length());
-        ReplicaSystem replica;
-        try {
-          replica = replicaSystemFactory.get(monitor.getContext(), property.getValue());
-        } catch (Exception e) {
-          log.warn("Could not instantiate ReplicaSystem for {} with configuration {}",
-              property.getKey(), property.getValue(), e);
-          continue;
-        }
-
-        peers.put(peerName, replica.getClass().getName());
-      }
-    }
-
-    final String targetPrefix = Property.TABLE_REPLICATION_TARGET.getKey();
-
-    // The total set of configured targets
-    Set<ReplicationTarget> allConfiguredTargets = new HashSet<>();
-
-    // Number of files per target we have to replicate
-    Map<ReplicationTarget,Long> targetCounts = new HashMap<>();
-
-    Map<String,TableId> tableNameToId = monitor.getContext().getTableNameToIdMap();
-    Map<TableId,String> tableIdToName = invert(tableNameToId);
-
-    for (String table : tops.list()) {
-      if (MetadataTable.NAME.equals(table) || RootTable.NAME.equals(table)) {
-        continue;
-      }
-      TableId localId = tableNameToId.get(table);
-      if (localId == null) {
-        log.trace("Could not determine ID for {}", table);
-        continue;
-      }
-
-      Map<String,String> propertiesForTable;
-      try {
-        propertiesForTable = tops.getConfiguration(table);
-      } catch (TableNotFoundException e) {
-        log.warn("Could not fetch properties for {}", table, e);
-        continue;
-      }
-      propertiesForTable.forEach((key, value) -> {
-        if (key.startsWith(targetPrefix)) {
-          String peerName = key.substring(targetPrefix.length());
-          String remoteIdentifier = value;
-          ReplicationTarget target = new ReplicationTarget(peerName, remoteIdentifier, localId);
-
-          allConfiguredTargets.add(target);
-        }
-      });
-    }
-
-    // Ensure replication table is online before attempting to create BatchScanner
-    if (!ReplicationTable.isOnline(client)) {
-      log.debug("Replication page requested, but replication table is offline");
-      return Collections.emptyList();
-    }
-
-    // Read over the queued work
-    BatchScanner bs;
-    try {
-      bs = client.createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4);
-    } catch (TableOfflineException | TableNotFoundException e) {
-      log.error("Could not read replication table", e);
-      return Collections.emptyList();
-    }
-
-    bs.setRanges(Collections.singleton(new Range()));
-    WorkSection.limit(bs);
-    try {
-      Text buffer = new Text();
-      for (Entry<Key,Value> entry : bs) {
-        Key k = entry.getKey();
-        k.getColumnQualifier(buffer);
-        ReplicationTarget target = ReplicationTarget.from(buffer);
-
-        // TODO ACCUMULO-2835 once explicit lengths are tracked, we can give size-based estimates
-        // instead of just file-based
-        Long count = targetCounts.get(target);
-        if (count == null) {
-          targetCounts.put(target, 1L);
-        } else {
-          targetCounts.put(target, count + 1);
-        }
-      }
-    } finally {
-      bs.close();
-    }
-
-    List<ReplicationInformation> replicationInformation = new ArrayList<>();
-    for (ReplicationTarget configuredTarget : allConfiguredTargets) {
-      String tableName = tableIdToName.get(configuredTarget.getSourceTableId());
-      if (tableName == null) {
-        log.trace("Could not determine table name from id {}", configuredTarget.getSourceTableId());
-        continue;
-      }
-
-      String replicaSystemClass = peers.get(configuredTarget.getPeerName());
-      if (replicaSystemClass == null) {
-        log.trace("Could not determine configured ReplicaSystem for {}",
-            configuredTarget.getPeerName());
-        continue;
-      }
-
-      Long numFiles = targetCounts.get(configuredTarget);
-
-      replicationInformation.add(new ReplicationInformation(tableName,
-          configuredTarget.getPeerName(), configuredTarget.getRemoteIdentifier(),
-          replicaSystemClass, (numFiles == null) ? 0 : numFiles));
-    }
-
-    return replicationInformation;
-  }
-
-  protected Map<TableId,String> invert(Map<String,TableId> map) {
-    Map<TableId,String> newMap = new HashMap<>(map.size());
-    for (Entry<String,TableId> entry : map.entrySet()) {
-      newMap.put(entry.getValue(), entry.getKey());
-    }
-    return newMap;
-  }
-}
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java
index 92093bd..5ff0f45 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/scans/ScansResource.java
@@ -27,11 +27,12 @@
 import jakarta.ws.rs.core.MediaType;
 
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.monitor.Monitor.ScanStats;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Generate a new Scan list JSON object
  *
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TableInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TableInformation.java
index 4599a62..e268cda 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TableInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TableInformation.java
@@ -21,7 +21,7 @@
 import jakarta.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.monitor.util.JaxbAbstractIdSerializer;
 
 /**
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
index 0536e63..65f6934 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tables/TablesResource.java
@@ -40,10 +40,9 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.monitor.Monitor;
@@ -142,11 +141,11 @@
     }
 
     TreeSet<String> locs = new TreeSet<>();
-    if (RootTable.ID.equals(tableId)) {
+    if (AccumuloTable.ROOT.tableId().equals(tableId)) {
       locs.add(rootTabletLocation);
     } else {
-      String systemTableName =
-          MetadataTable.ID.equals(tableId) ? RootTable.NAME : MetadataTable.NAME;
+      String systemTableName = AccumuloTable.METADATA.tableId().equals(tableId)
+          ? AccumuloTable.ROOT.tableName() : AccumuloTable.METADATA.tableName();
       MetaDataTableScanner scanner = new MetaDataTableScanner(monitor.getContext(),
           new Range(TabletsSection.encodeRow(tableId, new Text()),
               TabletsSection.encodeRow(tableId, null)),
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/RecoveryStatusInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/RecoveryStatusInformation.java
index 3f444ea..735acaf 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/RecoveryStatusInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/RecoveryStatusInformation.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.monitor.rest.trace;
 
-import org.apache.accumulo.core.master.thrift.RecoveryStatus;
+import org.apache.accumulo.core.manager.thrift.RecoveryStatus;
 
 /**
  * Generates a recovery status
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java
index abbd908..5383eb3 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServer.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.monitor.rest.tservers;
 
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 
 /**
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java
index 9000651..f31a154 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformation.java
@@ -23,9 +23,9 @@
 
 import jakarta.xml.bind.annotation.XmlAttribute;
 
-import org.apache.accumulo.core.master.thrift.RecoveryStatus;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.RecoveryStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.monitor.rest.tables.CompactionsList;
 import org.apache.accumulo.monitor.rest.tables.CompactionsTypes;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
index 820f347..c91fa6a 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
@@ -40,21 +40,22 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.RecoveryStatus;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.RecoveryStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.tabletserver.thrift.ActionStats;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 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.monitor.Monitor;
 import org.apache.accumulo.monitor.rest.manager.ManagerResource;
 import org.apache.accumulo.server.manager.state.DeadServerList;
 import org.apache.accumulo.server.util.ActionStatsUpdator;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Generates tserver lists as JSON objects
  *
@@ -124,7 +125,7 @@
     for (TabletServerStatus server : mmi.tServerInfo) {
       if (server.logSorts != null) {
         for (RecoveryStatus recovery : server.logSorts) {
-          String serv = AddressUtil.parseAddress(server.name, false).getHost();
+          String serv = AddressUtil.parseAddress(server.name).getHost();
           String log = recovery.name;
           int time = recovery.runtime;
           double progress = recovery.progress;
@@ -183,7 +184,7 @@
 
     try {
       ClientContext context = monitor.getContext();
-      TabletClientService.Client client =
+      TabletServerClientService.Client client =
           ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
       try {
         for (String tableId : mmi.tableMap.keySet()) {
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/logging/AccumuloMonitorAppender.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/logging/AccumuloMonitorAppender.java
index 7dfde0f..dce3917 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/logging/AccumuloMonitorAppender.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/logging/AccumuloMonitorAppender.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.monitor.util.logging;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -46,8 +47,6 @@
 import org.apache.logging.log4j.core.config.plugins.Plugin;
 import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory;
 
-import com.google.gson.Gson;
-
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
@@ -75,7 +74,6 @@
 
   }
 
-  private final Gson gson = new Gson();
   private final HttpClient httpClient = HttpClient.newHttpClient();
   private final Supplier<Optional<URI>> monitorLocator;
 
@@ -118,7 +116,7 @@
         pojo.message = event.getMessage().getFormattedMessage();
         pojo.stacktrace = throwableToStacktrace(event.getThrown());
 
-        String jsonEvent = gson.toJson(pojo);
+        String jsonEvent = GSON.get().toJson(pojo);
 
         var req = HttpRequest.newBuilder(uri).POST(BodyPublishers.ofString(jsonEvent, UTF_8))
             .setHeader("Content-Type", "application/json").build();
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
index cb69bf9..d508e09 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
@@ -344,23 +344,4 @@
     return model;
   }
 
-  /**
-   * Returns replication table template
-   *
-   * @return Replication model
-   */
-  @GET
-  @Path("replication")
-  @Template(name = "/default.ftl")
-  public Map<String,Object> getReplication() {
-
-    Map<String,Object> model = getModel();
-    model.put("title", "Replication Overview");
-
-    model.put("template", "replication.ftl");
-    model.put("js", "replication.js");
-
-    return model;
-  }
-
 }
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
index af0363a..11dbe5e 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
@@ -495,14 +495,6 @@
   return getJSONForTable('/rest/problems/details', 'problemDetails');
 }
 
-/**
- * REST GET call for the replication table,
- * stores it on a sessionStorage variable
- */
-function getReplication() {
-  return getJSONForTable('/rest/replication', 'replication');
-}
-
 //// Overview Plots Rest Calls
 
 /**
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/replication.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/replication.js
deleted file mode 100644
index bdfd2fe..0000000
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/replication.js
+++ /dev/null
@@ -1,79 +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.
- */
-"use strict";
-
-var replicationStatsTable;
-
-/**
- * Populates the table with the new information
- */
-function refreshReplication() {
-  ajaxReloadTable(replicationStatsTable);
-}
-
-/**
- * Used to redraw the page
- */
-function refresh() {
-  refreshReplication();
-}
-
-/**
- * Creates replication initial table
- */
-$(document).ready(function () {
-
-  replicationStatsTable = $('#replicationStats').DataTable({
-    "ajax": {
-      "url": "/rest/replication",
-      "dataSrc": ""
-    },
-    "stateSave": true,
-    "columns": [{
-        "data": "tableName",
-        "width": "25%"
-      },
-      {
-        "data": "peerName",
-        "width": "20%"
-      },
-      {
-        "data": "remoteIdentifier",
-        "width": "25%"
-      },
-      {
-        "data": "replicaSystemType",
-        "width": "15%"
-      },
-      {
-        "data": "filesNeedingReplication",
-        "width": "15%",
-        "render": function (data, type) {
-          if (type === 'display') {
-            data = bigNumberForQuantity(data);
-          }
-          return data;
-        }
-      }
-    ]
-  });
-
-  refreshReplication();
-
-});
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
index 5b325c2..09561ed 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
@@ -56,7 +56,6 @@
                 <li><a class="dropdown-item" href="/scans">Active&nbsp;Scans</a></li>
                 <li><a class="dropdown-item" href="/bulkImports">Bulk&nbsp;Imports</a></li>
                 <li><a class="dropdown-item" href="/ec">External&nbsp;Compactions</a></li>
-                <li><a class="dropdown-item" href="/replication">Replication</a></li>
               </ul>
             </li>
             <li class="dropdown">
@@ -90,4 +89,4 @@
           </ul>
         </div>
       </div>
-    </nav>
\ No newline at end of file
+    </nav>
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/replication.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/replication.ftl
deleted file mode 100644
index 407887f..0000000
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/replication.ftl
+++ /dev/null
@@ -1,42 +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.
-
--->
-      <div class="row">
-        <div class="col-xs-12">
-          <h3>${title}</h3>
-        </div>
-      </div>
-      <div class="row">
-        <div class="col-xs-12">
-          <table id="replicationStats" class="table caption-top table-bordered table-striped table-condensed">
-            <caption><span class="table-caption">Replication Status</span><br />
-            <thead>
-              <tr>
-                <th>Table&nbsp;</th>
-                <th>Peer&nbsp;</th>
-                <th>Remote&nbsp;Identifier&nbsp;</th>
-                <th>Replica&nbsp;System&nbsp;Type&nbsp;</th>
-                <th>Files&nbsp;needing&nbsp;replication&nbsp;</th>
-              </tr>
-            </thead>
-            <tbody></tbody>
-          </table>
-        </div>
-      </div>
diff --git a/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java b/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java
index 3d89793..66bc0be 100644
--- a/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java
+++ b/server/monitor/src/test/java/org/apache/accumulo/monitor/rest/tservers/TabletServerInformationTest.java
@@ -23,11 +23,11 @@
 
 import java.util.Collections;
 
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
-import org.apache.accumulo.core.master.thrift.Compacting;
-import org.apache.accumulo.core.master.thrift.RecoveryStatus;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
+import org.apache.accumulo.core.manager.thrift.Compacting;
+import org.apache.accumulo.core.manager.thrift.RecoveryStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.monitor.rest.tables.CompactionsTypes;
 import org.apache.accumulo.monitor.rest.trace.RecoveryStatusInformation;
diff --git a/server/native/pom.xml b/server/native/pom.xml
index 74a8be0..1ac0de4 100644
--- a/server/native/pom.xml
+++ b/server/native/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-native</artifactId>
@@ -46,11 +46,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.junit.vintage</groupId>
-      <artifactId>junit-vintage-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <scope>test</scope>
diff --git a/server/tserver/pom.xml b/server/tserver/pom.xml
index a22eb52..911f0e0 100644
--- a/server/tserver/pom.xml
+++ b/server/tserver/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>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-tserver</artifactId>
@@ -49,10 +49,6 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
index 8b4f117..552d9f4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
@@ -156,9 +156,7 @@
     Tablet tablet = null;
     boolean successful = false;
 
-    try {
-      server.acquireRecoveryMemory(extent);
-
+    try (var recoveryMemory = server.acquireRecoveryMemory(tabletMetadata)) {
       TabletResourceManager trm = server.resourceManager.createTabletResourceManager(extent,
           server.getTableConfiguration(extent));
       TabletData data = new TabletData(tabletMetadata);
@@ -205,8 +203,6 @@
       TableId tableId = extent.tableId();
       ProblemReports.getInstance(server.getContext()).report(new ProblemReport(tableId, TABLET_LOAD,
           extent.getUUID().toString(), server.getClientAddressString(), e));
-    } finally {
-      server.releaseRecoveryMemory(extent);
     }
 
     if (successful) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java
deleted file mode 100644
index 6870f45..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java
+++ /dev/null
@@ -1,82 +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.tserver;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue.Processor;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Copy failed bulk imports.
- */
-// TODO: Remove when Property.TSERV_WORKQ_THREADS is removed
-public class BulkFailedCopyProcessor implements Processor {
-
-  private static final Logger log = LoggerFactory.getLogger(BulkFailedCopyProcessor.class);
-
-  private ServerContext context;
-
-  BulkFailedCopyProcessor(ServerContext context) {
-    this.context = context;
-  }
-
-  @Override
-  public Processor newProcessor() {
-    return new BulkFailedCopyProcessor(context);
-  }
-
-  @Override
-  public void process(String workID, byte[] data) {
-
-    String[] paths = new String(data, UTF_8).split(",");
-
-    Path orig = new Path(paths[0]);
-    Path dest = new Path(paths[1]);
-    Path tmp = new Path(dest.getParent(), dest.getName() + ".tmp");
-
-    VolumeManager vm = context.getVolumeManager();
-    try {
-      FileSystem origFs = vm.getFileSystemByPath(orig);
-      FileSystem destFs = vm.getFileSystemByPath(dest);
-
-      FileUtil.copy(origFs, orig, destFs, tmp, false, true, context.getHadoopConf());
-      destFs.rename(tmp, dest);
-      log.debug("copied {} to {}", orig, dest);
-    } catch (IOException ex) {
-      try {
-        FileSystem destFs = vm.getFileSystemByPath(dest);
-        destFs.create(dest).close();
-        log.warn(" marked " + dest + " failed", ex);
-      } catch (IOException e) {
-        log.error("Unable to create failure flag file " + dest, e);
-      }
-    }
-
-  }
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
index a9c425d..2479c4a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.tserver;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -63,6 +63,7 @@
 import org.apache.accumulo.core.iteratorsImpl.system.SortedMapIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator.DataSource;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.sample.impl.SamplerFactory;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
@@ -602,7 +603,8 @@
 
         TableConfiguration tableConf = context.getTableConfiguration(tableId);
         reader = new RFileOperations().newReaderBuilder()
-            .forFile(memDumpFile, fs, conf, tableConf.getCryptoService())
+            .forFile(UnreferencedTabletFile.of(fs, new Path(memDumpFile)), fs, conf,
+                tableConf.getCryptoService())
             .withTableConfiguration(tableConf).seekToBeginning().build();
         if (iflag != null) {
           reader.setInterruptFlag(iflag);
@@ -783,7 +785,8 @@
 
         TableConfiguration tableConf = context.getTableConfiguration(tableId);
         FileSKVWriter out = new RFileOperations().newWriterBuilder()
-            .forFile(tmpFile, fs, newConf, tableConf.getCryptoService())
+            .forFile(UnreferencedTabletFile.of(fs, new Path(tmpFile)), fs, newConf,
+                tableConf.getCryptoService())
             .withTableConfiguration(aconf).build();
 
         InterruptibleIterator iter = map.skvIterator(null);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
index 432d7c5..da2528e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
@@ -18,8 +18,7 @@
  */
 package org.apache.accumulo.tserver;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
@@ -34,6 +33,7 @@
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
@@ -46,7 +46,9 @@
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -59,43 +61,46 @@
 import org.apache.accumulo.core.dataImpl.thrift.TColumn;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.spi.scan.ScanServerSelector;
-import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
+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.TSamplerConfiguration;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.client.ClientServiceHandler;
+import org.apache.accumulo.server.compaction.PausedCompactionMetrics;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.rpc.ServerAddress;
 import org.apache.accumulo.server.rpc.TServerUtils;
 import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
 import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
 import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
 import org.apache.accumulo.tserver.session.MultiScanSession;
@@ -114,7 +119,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
 import com.github.benmanes.caffeine.cache.CacheLoader;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
@@ -122,24 +126,13 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+import com.google.common.net.HostAndPort;
 
 import io.micrometer.core.instrument.Tag;
 
 public class ScanServer extends AbstractServer
     implements TabletScanClientService.Iface, TabletHostingServer {
 
-  public static class ScanServerOpts extends ServerOpts {
-    @Parameter(required = false, names = {"-g", "--group"},
-        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
-            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
-            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
-    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
-
-    public String getGroupName() {
-      return groupName;
-    }
-  }
-
   private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
 
   private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
@@ -162,10 +155,15 @@
     @Override
     public Map<? extends KeyExtent,? extends TabletMetadata>
         loadAll(Set<? extends KeyExtent> keys) {
+      Map<KeyExtent,TabletMetadata> tms;
       long t1 = System.currentTimeMillis();
       @SuppressWarnings("unchecked")
-      var tms = ample.readTablets().forTablets((Collection<KeyExtent>) keys).build().stream()
-          .collect(Collectors.toMap(tm -> tm.getExtent(), tm -> tm));
+      Collection<KeyExtent> extents = (Collection<KeyExtent>) keys;
+      try (TabletsMetadata tabletsMetadata =
+          ample.readTablets().forTablets(extents, Optional.empty()).build()) {
+        tms =
+            tabletsMetadata.stream().collect(Collectors.toMap(TabletMetadata::getExtent, tm -> tm));
+      }
       long t2 = System.currentTimeMillis();
       LOG.trace("Read metadata for {} tablets in {} ms", keys.size(), t2 - t1);
       return tms;
@@ -197,7 +195,6 @@
   private final SessionManager sessionManager;
   private final TabletServerResourceManager resourceManager;
   HostAndPort clientAddress;
-  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
 
   private volatile boolean serverStopRequested = false;
   private ServiceLock scanServerLock;
@@ -209,7 +206,7 @@
 
   private final String groupName;
 
-  public ScanServer(ScanServerOpts opts, String[] args) {
+  public ScanServer(ConfigOpts opts, String[] args) {
     super("sserver", opts, args);
 
     context = super.getContext();
@@ -253,7 +250,7 @@
 
     delegate = newThriftScanClientHandler(new WriteTracker());
 
-    this.groupName = Objects.requireNonNull(opts.getGroupName());
+    this.groupName = getConfiguration().get(Property.SSERV_GROUP_NAME);
 
     ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
         .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
@@ -277,7 +274,10 @@
 
     // This class implements TabletClientService.Iface and then delegates calls. Be sure
     // to set up the ThriftProcessor using this class, not the delegate.
-    TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext());
+    ClientServiceHandler clientHandler =
+        new ClientServiceHandler(context, new TransactionWatcher(context));
+    TProcessor processor =
+        ThriftProcessorTypes.getScanServerTProcessor(clientHandler, this, getContext());
 
     Property maxMessageSizeProperty =
         (getConfiguration().get(Property.SSERV_MAX_MESSAGE_SIZE) != null
@@ -330,7 +330,7 @@
             if (!serverStopRequested) {
               LOG.error("Lost tablet server lock (reason = {}), exiting.", reason);
             }
-            gcLogger.logGCInfo(getConfiguration());
+            context.getLowMemoryDetector().logGCInfo(getConfiguration());
           });
         }
 
@@ -340,14 +340,17 @@
         }
       };
 
-      // Don't use the normal ServerServices lock content, instead put the server UUID here.
-      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
-
-      // wait for 120 seconds with 5 second delay
       for (int i = 0; i < 120 / 5; i++) {
         zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
 
-        if (scanServerLock.tryLock(lw, lockContent)) {
+        ServiceDescriptors descriptors = new ServiceDescriptors();
+        for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT,
+            ThriftService.TABLET_SCAN}) {
+          descriptors.addService(
+              new ServiceDescriptor(serverLockUUID, svc, getClientAddressString(), this.groupName));
+        }
+
+        if (scanServerLock.tryLock(lw, new ServiceLockData(descriptors))) {
           LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
           return scanServerLock;
         }
@@ -384,7 +387,7 @@
     blockCacheMetrics = new BlockCacheMetrics(resourceManager.getIndexCache(),
         resourceManager.getDataCache(), resourceManager.getSummaryCache());
 
-    metricsInfo.addMetricsProducers(scanMetrics, scanServerMetrics, blockCacheMetrics);
+    metricsInfo.addMetricsProducers(this, scanMetrics, scanServerMetrics, blockCacheMetrics);
     metricsInfo.init();
     // We need to set the compaction manager so that we don't get an NPE in CompactableImpl.close
 
@@ -412,7 +415,7 @@
         LOG.warn("Failed to close filesystem : {}", e.getMessage(), e);
       }
 
-      gcLogger.logGCInfo(getConfiguration());
+      context.getLowMemoryDetector().logGCInfo(getConfiguration());
       LOG.info("stop requested. exiting ... ");
       try {
         if (null != lock) {
@@ -593,7 +596,8 @@
 
       for (StoredTabletFile file : allFiles.keySet()) {
         if (!reservedFiles.containsKey(file)) {
-          refs.add(new ScanServerRefTabletFile(file.getPathStr(), serverAddress, serverLockUUID));
+          refs.add(new ScanServerRefTabletFile(file.getNormalizedPathStr(), serverAddress,
+              serverLockUUID));
           filesToReserve.add(file);
           tabletsToCheck.add(Objects.requireNonNull(allFiles.get(file)));
           LOG.trace("RFFS {} need to add scan ref for file {}", myReservationId, file);
@@ -796,8 +800,8 @@
             // then adding the file to influxFiles will make it wait until we finish
             influxFiles.add(file);
             confirmed.add(file);
-            refsToDelete
-                .add(new ScanServerRefTabletFile(file.getPathStr(), serverAddress, serverLockUUID));
+            refsToDelete.add(new ScanServerRefTabletFile(file.getNormalizedPathStr(), serverAddress,
+                serverLockUUID));
 
             // remove the entry from the map while holding the write lock ensuring no new
             // reservations are added to the map values while the metadata operation to delete is
@@ -1043,6 +1047,12 @@
   }
 
   @Override
+  public PausedCompactionMetrics getPausedCompactionMetrics() {
+    // ScanServer does not perform compactions
+    return null;
+  }
+
+  @Override
   public Session getSession(long scanID) {
     return sessionManager.getSession(scanID);
   }
@@ -1063,17 +1073,12 @@
   }
 
   @Override
-  public GarbageCollectionLogger getGcLogger() {
-    return gcLogger;
-  }
-
-  @Override
   public BlockCacheConfiguration getBlockCacheConfiguration(AccumuloConfiguration acuConf) {
     return BlockCacheConfiguration.forScanServer(acuConf);
   }
 
   public static void main(String[] args) throws Exception {
-    try (ScanServer tserver = new ScanServer(new ScanServerOpts(), args)) {
+    try (ScanServer tserver = new ScanServer(new ConfigOpts(), args)) {
       tserver.runServer();
     }
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
index acd0a2f..af16bee 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletClientHandler.java
@@ -50,6 +50,7 @@
 import org.apache.accumulo.core.clientImpl.DurabilityImpl;
 import org.apache.accumulo.core.clientImpl.TabletType;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
@@ -59,7 +60,6 @@
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.dataImpl.thrift.TCMResult;
 import org.apache.accumulo.core.dataImpl.thrift.TCMStatus;
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalMutation;
@@ -70,15 +70,14 @@
 import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
 import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.logging.TabletLogger;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
@@ -86,22 +85,25 @@
 import org.apache.accumulo.core.summary.Gatherer;
 import org.apache.accumulo.core.summary.Gatherer.FileSystemResolver;
 import org.apache.accumulo.core.summary.SummaryCollection;
+import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
+import org.apache.accumulo.core.tabletingest.thrift.TDurability;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
-import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
-import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 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.Halt;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.compaction.CompactionInfo;
 import org.apache.accumulo.server.compaction.FileCompactor;
@@ -131,16 +133,17 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.Cache;
+import com.github.benmanes.caffeine.cache.Cache;
 
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
-public class TabletClientHandler implements TabletClientService.Iface {
+public class TabletClientHandler implements TabletServerClientService.Iface,
+    TabletIngestClientService.Iface, TabletManagementClientService.Iface {
 
   private static final Logger log = LoggerFactory.getLogger(TabletClientHandler.class);
   private final long MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS;
-  private static final long RECENTLY_SPLIT_MILLIES = MINUTES.toMillis(1);
+  private static final long RECENTLY_SPLIT_MILLIS = MINUTES.toMillis(1);
   private final TabletServer server;
   protected final TransactionWatcher watcher;
   protected final ServerContext context;
@@ -161,56 +164,8 @@
   }
 
   @Override
-  public List<TKeyExtent> bulkImport(TInfo tinfo, TCredentials credentials, final long tid,
-      final Map<TKeyExtent,Map<String,MapFileInfo>> files, final boolean setTime)
-      throws ThriftSecurityException {
-
-    if (!security.canPerformSystemActions(credentials)) {
-      throw new ThriftSecurityException(credentials.getPrincipal(),
-          SecurityErrorCode.PERMISSION_DENIED);
-    }
-
-    try {
-      return watcher.run(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
-        List<TKeyExtent> failures = new ArrayList<>();
-
-        for (Entry<TKeyExtent,Map<String,MapFileInfo>> entry : files.entrySet()) {
-          TKeyExtent tke = entry.getKey();
-          Map<String,MapFileInfo> fileMap = entry.getValue();
-          Map<TabletFile,MapFileInfo> fileRefMap = new HashMap<>();
-          for (Entry<String,MapFileInfo> mapping : fileMap.entrySet()) {
-            Path path = new Path(mapping.getKey());
-            FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
-            path = ns.makeQualified(path);
-            fileRefMap.put(new TabletFile(path), mapping.getValue());
-          }
-
-          Tablet importTablet = server.getOnlineTablet(KeyExtent.fromThrift(tke));
-
-          if (importTablet == null) {
-            failures.add(tke);
-          } else {
-            try {
-              importTablet.importMapFiles(tid, fileRefMap, setTime);
-            } catch (IOException ioe) {
-              log.info("files {} not imported to {}: {}", fileMap.keySet(),
-                  KeyExtent.fromThrift(tke), ioe.getMessage());
-              failures.add(tke);
-            }
-          }
-        }
-        return failures;
-      });
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
   public void loadFiles(TInfo tinfo, TCredentials credentials, long tid, String dir,
-      Map<TKeyExtent,Map<String,MapFileInfo>> tabletImports, boolean setTime)
+      Map<TKeyExtent,Map<String,DataFileInfo>> tabletImports, boolean setTime)
       throws ThriftSecurityException {
     if (!security.canPerformSystemActions(credentials)) {
       throw new ThriftSecurityException(credentials.getPrincipal(),
@@ -219,15 +174,16 @@
 
     watcher.runQuietly(Constants.BULK_ARBITRATOR_TYPE, tid, () -> {
       tabletImports.forEach((tke, fileMap) -> {
-        Map<TabletFile,MapFileInfo> newFileMap = new HashMap<>();
+        Map<ReferencedTabletFile,DataFileInfo> newFileMap = new HashMap<>();
 
-        for (Entry<String,MapFileInfo> mapping : fileMap.entrySet()) {
+        for (Entry<String,DataFileInfo> mapping : fileMap.entrySet()) {
           Path path = new Path(dir, mapping.getKey());
           FileSystem ns = context.getVolumeManager().getFileSystemByPath(path);
           path = ns.makeQualified(path);
-          newFileMap.put(new TabletFile(path), mapping.getValue());
+          newFileMap.put(new ReferencedTabletFile(path), mapping.getValue());
         }
-        var files = newFileMap.keySet().stream().map(TabletFile::getPathStr).collect(toList());
+        var files = newFileMap.keySet().stream().map(ReferencedTabletFile::getNormalizedPathStr)
+            .collect(toList());
         server.updateBulkImportState(files, BulkImportState.INITIAL);
 
         Tablet importTablet = server.getOnlineTablet(KeyExtent.fromThrift(tke));
@@ -235,7 +191,7 @@
         if (importTablet != null) {
           try {
             server.updateBulkImportState(files, BulkImportState.PROCESSING);
-            importTablet.importMapFiles(tid, newFileMap, setTime);
+            importTablet.importDataFiles(tid, newFileMap, setTime);
           } catch (IOException ioe) {
             log.debug("files {} not imported to {}: {}", fileMap.keySet(),
                 KeyExtent.fromThrift(tke), ioe.getMessage());
@@ -308,7 +264,6 @@
       us.currentTablet = null;
       us.authFailures.put(keyExtent, SecurityErrorCode.TABLE_DOESNT_EXIST);
       server.updateMetrics.addUnknownTabletErrors(0);
-      return;
     } catch (ThriftSecurityException e) {
       log.error("Denying permission to check user " + us.getUser() + " with user " + e.getUser(),
           e);
@@ -317,7 +272,6 @@
       us.currentTablet = null;
       us.authFailures.put(keyExtent, e.getCode());
       server.updateMetrics.addPermissionErrors(0);
-      return;
     }
   }
 
@@ -584,7 +538,8 @@
           us.authFailures.entrySet().stream()
               .collect(Collectors.toMap(e -> e.getKey().toThrift(), Entry::getValue)));
     } finally {
-      // Atomically unreserve and delete the session. If there any write stragglers, they will fail
+      // Atomically unreserve and delete the session. If there are any write stragglers, they will
+      // fail
       // after this point.
       server.sessionManager.removeSession(updateID, true);
     }
@@ -595,96 +550,6 @@
     return server.sessionManager.removeIfNotReserved(updateID);
   }
 
-  @Override
-  public void update(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent,
-      TMutation tmutation, TDurability tdurability)
-      throws NotServingTabletException, ConstraintViolationException, ThriftSecurityException {
-
-    final TableId tableId = TableId.of(new String(tkeyExtent.getTable(), UTF_8));
-    NamespaceId namespaceId = getNamespaceId(credentials, tableId);
-    if (!security.canWrite(credentials, tableId, namespaceId)) {
-      throw new ThriftSecurityException(credentials.getPrincipal(),
-          SecurityErrorCode.PERMISSION_DENIED);
-    }
-    final KeyExtent keyExtent = KeyExtent.fromThrift(tkeyExtent);
-    final Tablet tablet = server.getOnlineTablet(KeyExtent.copyOf(keyExtent));
-    if (tablet == null) {
-      throw new NotServingTabletException(tkeyExtent);
-    }
-    Durability tabletDurability = tablet.getDurability();
-
-    if (!keyExtent.isMeta()) {
-      try {
-        server.resourceManager.waitUntilCommitsAreEnabled();
-      } catch (HoldTimeoutException hte) {
-        // Major hack. Assumption is that the client has timed out and is gone. If that's not the
-        // case, then throwing the following will let client know there
-        // was a failure and it should retry.
-        throw new NotServingTabletException(tkeyExtent);
-      }
-    }
-
-    final long opid = writeTracker.startWrite(TabletType.type(keyExtent));
-
-    try {
-      final Mutation mutation = new ServerMutation(tmutation);
-      final List<Mutation> mutations = Collections.singletonList(mutation);
-
-      PreparedMutations prepared;
-      Span span = TraceUtil.startSpan(this.getClass(), "update::prep");
-      try (Scope scope = span.makeCurrent()) {
-        prepared = tablet.prepareMutationsForCommit(
-            new TservConstraintEnv(server.getContext(), security, credentials), mutations);
-      } catch (Exception e) {
-        TraceUtil.setException(span, e, true);
-        throw e;
-      } finally {
-        span.end();
-      }
-
-      if (prepared.tabletClosed()) {
-        throw new NotServingTabletException(tkeyExtent);
-      } else if (!prepared.getViolators().isEmpty()) {
-        throw new ConstraintViolationException(prepared.getViolations().asList().stream()
-            .map(ConstraintViolationSummary::toThrift).collect(Collectors.toList()));
-      } else {
-        CommitSession session = prepared.getCommitSession();
-        Durability durability = DurabilityImpl
-            .resolveDurabilty(DurabilityImpl.fromThrift(tdurability), tabletDurability);
-
-        // Instead of always looping on true, skip completely when durability is NONE.
-        while (durability != Durability.NONE) {
-          try {
-            Span span2 = TraceUtil.startSpan(this.getClass(), "update::wal");
-            try (Scope scope = span2.makeCurrent()) {
-              server.logger.log(session, mutation, durability);
-            } catch (Exception e) {
-              TraceUtil.setException(span2, e, true);
-              throw e;
-            } finally {
-              span2.end();
-            }
-            break;
-          } catch (IOException ex) {
-            log.warn("Error writing mutations to log", ex);
-          }
-        }
-
-        Span span3 = TraceUtil.startSpan(this.getClass(), "update::commit");
-        try (Scope scope = span3.makeCurrent()) {
-          session.commit(mutations);
-        } catch (Exception e) {
-          TraceUtil.setException(span3, e, true);
-          throw e;
-        } finally {
-          span3.end();
-        }
-      }
-    } finally {
-      writeTracker.finishWrite(opid);
-    }
-  }
-
   private NamespaceId getNamespaceId(TCredentials credentials, TableId tableId)
       throws ThriftSecurityException {
     try {
@@ -896,7 +761,7 @@
       String classLoaderContext) throws ThriftSecurityException, TException {
 
     TableId tableId = TableId.of(tableIdStr);
-    Authorizations userauths = null;
+    Authorizations userauths;
     NamespaceId namespaceId = getNamespaceId(credentials, tableId);
     if (!security.canConditionallyUpdate(credentials, tableId, namespaceId)) {
       throw new ThriftSecurityException(credentials.getPrincipal(),
@@ -929,7 +794,8 @@
       throw new NoSuchScanIDException();
     }
 
-    if (!cs.tableId.equals(MetadataTable.ID) && !cs.tableId.equals(RootTable.ID)) {
+    if (!cs.tableId.equals(AccumuloTable.METADATA.tableId())
+        && !cs.tableId.equals(AccumuloTable.ROOT.tableId())) {
       try {
         server.resourceManager.waitUntilCommitsAreEnabled();
       } catch (HoldTimeoutException hte) {
@@ -969,7 +835,7 @@
     } catch (IOException ioe) {
       throw new TException(ioe);
     } catch (Exception e) {
-      log.warn("Exception returned for conditionalUpdate {}", e);
+      log.warn("Exception returned for conditionalUpdate. tableId: {}, opid: {}", tid, opid, e);
       throw e;
     } finally {
       writeTracker.finishWrite(opid);
@@ -1089,7 +955,7 @@
       Halt.halt(1, () -> {
         log.info("Tablet server no longer holds lock during checkPermission() : {}, exiting",
             request);
-        server.getGcLogger().logGCInfo(server.getConfiguration());
+        context.getLowMemoryDetector().logGCInfo(server.getConfiguration());
       });
     }
 
@@ -1152,7 +1018,7 @@
             for (KeyExtent e2 : onlineOverlapping) {
               Tablet tablet = server.getOnlineTablet(e2);
               if (System.currentTimeMillis() - tablet.getSplitCreationTime()
-                  < RECENTLY_SPLIT_MILLIES) {
+                  < RECENTLY_SPLIT_MILLIS) {
                 all.remove(e2);
               }
             }
@@ -1210,8 +1076,8 @@
 
     KeyExtent extent = KeyExtent.fromThrift(textent);
 
-    server.resourceManager.addMigration(extent,
-        new UnloadTabletHandler(server, extent, goal, requestTime));
+    server.resourceManager.addMigration(extent, new UnloadTabletHandler(server, extent, goal,
+        SteadyTime.from(requestTime, TimeUnit.MILLISECONDS)));
   }
 
   @Override
@@ -1277,7 +1143,7 @@
 
     Halt.halt(0, () -> {
       log.info("Manager requested tablet server halt");
-      server.gcLogger.logGCInfo(server.getConfiguration());
+      context.getLowMemoryDetector().logGCInfo(server.getConfiguration());
       server.requestStop();
       try {
         server.getLock().unlock();
@@ -1302,23 +1168,6 @@
   }
 
   @Override
-  public void chop(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent textent) {
-    try {
-      checkPermission(security, context, server, credentials, lock, "chop");
-    } catch (ThriftSecurityException e) {
-      log.error("Caller doesn't have permission to chop extent", e);
-      throw new RuntimeException(e);
-    }
-
-    KeyExtent ke = KeyExtent.fromThrift(textent);
-
-    Tablet tablet = server.getOnlineTablet(ke);
-    if (tablet != null) {
-      tablet.chopFiles();
-    }
-  }
-
-  @Override
   public void compact(TInfo tinfo, TCredentials credentials, String lock, String tableId,
       ByteBuffer startRow, ByteBuffer endRow) {
     try {
@@ -1434,12 +1283,9 @@
 
   @Override
   public List<String> getActiveLogs(TInfo tinfo, TCredentials credentials) {
-    String log = server.logger.getLogFile();
-    // Might be null if there no active logger
-    if (log == null) {
-      return Collections.emptyList();
-    }
-    return Collections.singletonList(log);
+    // Might be null if there is no active logger
+    LogEntry le = server.logger.getLogEntry();
+    return le == null ? Collections.emptyList() : Collections.singletonList(le.getPath());
   }
 
   @Override
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletHostingServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletHostingServer.java
index 3715eac..8e5047b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletHostingServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletHostingServer.java
@@ -20,12 +20,12 @@
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 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.spi.cache.BlockCacheManager;
 import org.apache.accumulo.core.spi.scan.ScanServerInfo;
-import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.compaction.PausedCompactionMetrics;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
 import org.apache.accumulo.tserver.session.Session;
@@ -50,6 +50,8 @@
 
   TabletServerScanMetrics getScanMetrics();
 
+  PausedCompactionMetrics getPausedCompactionMetrics();
+
   Session getSession(long scanID);
 
   TableConfiguration getTableConfiguration(KeyExtent threadPoolExtent);
@@ -58,7 +60,5 @@
 
   ZooCache getManagerLockCache();
 
-  GarbageCollectionLogger getGcLogger();
-
   BlockCacheManager.Configuration getBlockCacheConfiguration(AccumuloConfiguration acuConf);
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 85b0872..0590fb8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -18,12 +18,12 @@
  */
 package org.apache.accumulo.tserver;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.ECOMP;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS;
 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.threads.ThreadPools.watchCriticalFixedDelay;
 import static org.apache.accumulo.core.util.threads.ThreadPools.watchCriticalScheduledTask;
 import static org.apache.accumulo.core.util.threads.ThreadPools.watchNonCriticalScheduledTask;
@@ -31,7 +31,6 @@
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.UnknownHostException;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.time.Instant;
 import java.util.ArrayList;
@@ -46,6 +45,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
@@ -56,7 +56,6 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -64,6 +63,7 @@
 import java.util.function.Consumer;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.clientImpl.DurabilityImpl;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
@@ -72,21 +72,25 @@
 import org.apache.accumulo.core.data.InstanceId;
 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.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptors;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.Compacting;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.master.thrift.Compacting;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.rpc.ThriftUtil;
@@ -96,23 +100,18 @@
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.ComparablePair;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.Retry.RetryFactory;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.compaction.CompactionWatcher;
+import org.apache.accumulo.server.compaction.PausedCompactionMetrics;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -126,10 +125,8 @@
 import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.server.security.delegation.ZooAuthenticationKeyWatcher;
-import org.apache.accumulo.server.util.FileSystemMonitor;
 import org.apache.accumulo.server.util.ServerBulkImportStatus;
 import org.apache.accumulo.server.util.time.RelativeTime;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
 import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
@@ -166,18 +163,16 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterators;
+import com.google.common.net.HostAndPort;
 
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
 public class TabletServer extends AbstractServer implements TabletHostingServer {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Logger log = LoggerFactory.getLogger(TabletServer.class);
-  private static final long TIME_BETWEEN_GC_CHECKS = TimeUnit.SECONDS.toMillis(5);
   private static final long TIME_BETWEEN_LOCATOR_CACHE_CLEARS = TimeUnit.HOURS.toMillis(1);
 
-  final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
   final ZooCache managerLockCache;
 
   final TabletServerLogger logger;
@@ -187,6 +182,7 @@
   TabletServerScanMetrics scanMetrics;
   TabletServerMinCMetrics mincMetrics;
   CompactionExecutorsMetrics ceMetrics;
+  PausedCompactionMetrics pausedMetrics;
   BlockCacheMetrics blockCacheMetrics;
 
   @Override
@@ -198,9 +194,12 @@
     return mincMetrics;
   }
 
+  @Override
+  public PausedCompactionMetrics getPausedCompactionMetrics() {
+    return pausedMetrics;
+  }
+
   private final LogSorter logSorter;
-  @SuppressWarnings("deprecation")
-  private org.apache.accumulo.tserver.replication.ReplicationWorker replWorker = null;
   final TabletStatsKeeper statsKeeper;
   private final AtomicInteger logIdGenerator = new AtomicInteger();
 
@@ -225,7 +224,6 @@
   private ServiceLock tabletServerLock;
 
   private TServer server;
-  private volatile TServer replServer;
 
   private String lockID;
   private volatile long lockSessionId = -1;
@@ -239,12 +237,12 @@
   private final ServerContext context;
 
   public static void main(String[] args) throws Exception {
-    try (TabletServer tserver = new TabletServer(new ServerOpts(), args)) {
+    try (TabletServer tserver = new TabletServer(new ConfigOpts(), args)) {
       tserver.runServer();
     }
   }
 
-  protected TabletServer(ServerOpts opts, String[] args) {
+  protected TabletServer(ConfigOpts opts, String[] args) {
     super("tserver", opts, args);
     context = super.getContext();
     this.managerLockCache = new ZooCache(context.getZooReader(), null);
@@ -253,9 +251,6 @@
     log.info("Instance " + getInstanceID());
     this.sessionManager = new SessionManager(context);
     this.logSorter = new LogSorter(context, aconf);
-    @SuppressWarnings("deprecation")
-    var replWorker = new org.apache.accumulo.tserver.replication.ReplicationWorker(context);
-    this.replWorker = replWorker;
     this.statsKeeper = new TabletStatsKeeper();
     final int numBusyTabletsToLog = aconf.getCount(Property.TSERV_LOG_BUSY_TABLETS_COUNT);
     final long logBusyTabletsDelay =
@@ -309,12 +304,8 @@
         }), 5, 5, TimeUnit.SECONDS);
     watchNonCriticalScheduledTask(future);
 
-    @SuppressWarnings("deprecation")
-    final long walMaxSize =
-        aconf.getAsBytes(aconf.resolve(Property.TSERV_WAL_MAX_SIZE, Property.TSERV_WALOG_MAX_SIZE));
-    @SuppressWarnings("deprecation")
-    final long walMaxAge = aconf
-        .getTimeInMillis(aconf.resolve(Property.TSERV_WAL_MAX_AGE, Property.TSERV_WALOG_MAX_AGE));
+    final long walMaxSize = aconf.getAsBytes(Property.TSERV_WAL_MAX_SIZE);
+    final long walMaxAge = aconf.getTimeInMillis(Property.TSERV_WAL_MAX_AGE);
     final long minBlockSize =
         context.getHadoopConf().getLong("dfs.namenode.fs-limits.min-block-size", 0);
     if (minBlockSize != 0 && minBlockSize > walMaxSize) {
@@ -324,31 +315,25 @@
           + " or decrease dfs.namenode.fs-limits.min-block-size in hdfs-site.xml.");
     }
 
-    @SuppressWarnings("deprecation")
     final long toleratedWalCreationFailures =
-        aconf.getCount(aconf.resolve(Property.TSERV_WAL_TOLERATED_CREATION_FAILURES,
-            Property.TSERV_WALOG_TOLERATED_CREATION_FAILURES));
-    @SuppressWarnings("deprecation")
+        aconf.getCount(Property.TSERV_WAL_TOLERATED_CREATION_FAILURES);
     final long walFailureRetryIncrement =
-        aconf.getTimeInMillis(aconf.resolve(Property.TSERV_WAL_TOLERATED_WAIT_INCREMENT,
-            Property.TSERV_WALOG_TOLERATED_WAIT_INCREMENT));
-    @SuppressWarnings("deprecation")
+        aconf.getTimeInMillis(Property.TSERV_WAL_TOLERATED_WAIT_INCREMENT);
     final long walFailureRetryMax =
-        aconf.getTimeInMillis(aconf.resolve(Property.TSERV_WAL_TOLERATED_MAXIMUM_WAIT_DURATION,
-            Property.TSERV_WALOG_TOLERATED_MAXIMUM_WAIT_DURATION));
+        aconf.getTimeInMillis(Property.TSERV_WAL_TOLERATED_MAXIMUM_WAIT_DURATION);
     final RetryFactory walCreationRetryFactory =
         Retry.builder().maxRetries(toleratedWalCreationFailures)
-            .retryAfter(walFailureRetryIncrement, TimeUnit.MILLISECONDS)
-            .incrementBy(walFailureRetryIncrement, TimeUnit.MILLISECONDS)
-            .maxWait(walFailureRetryMax, TimeUnit.MILLISECONDS).backOffFactor(1.5)
-            .logInterval(3, TimeUnit.MINUTES).createFactory();
+            .retryAfter(Duration.ofMillis(walFailureRetryIncrement))
+            .incrementBy(Duration.ofMillis(walFailureRetryIncrement))
+            .maxWait(Duration.ofMillis(walFailureRetryMax)).backOffFactor(1.5)
+            .logInterval(Duration.ofMinutes(3)).createFactory();
     // Tolerate infinite failures for the write, however backing off the same as for creation
     // failures.
-    final RetryFactory walWritingRetryFactory = Retry.builder().infiniteRetries()
-        .retryAfter(walFailureRetryIncrement, TimeUnit.MILLISECONDS)
-        .incrementBy(walFailureRetryIncrement, TimeUnit.MILLISECONDS)
-        .maxWait(walFailureRetryMax, TimeUnit.MILLISECONDS).backOffFactor(1.5)
-        .logInterval(3, TimeUnit.MINUTES).createFactory();
+    final RetryFactory walWritingRetryFactory =
+        Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(walFailureRetryIncrement))
+            .incrementBy(Duration.ofMillis(walFailureRetryIncrement))
+            .maxWait(Duration.ofMillis(walFailureRetryMax)).backOffFactor(1.5)
+            .logInterval(Duration.ofMinutes(3)).createFactory();
 
     logger = new TabletServerLogger(this, walMaxSize, syncCounter, flushCounter,
         walCreationRetryFactory, walWritingRetryFactory, walMaxAge);
@@ -381,7 +366,7 @@
 
   private static long jitter() {
     // add a random 10% wait
-    return (long) ((1. + (random.nextDouble() / 10))
+    return (long) ((1. + (RANDOM.get().nextDouble() / 10))
         * TabletServer.TIME_BETWEEN_LOCATOR_CACHE_CLEARS);
   }
 
@@ -532,15 +517,14 @@
     managerMessages.addLast(m);
   }
 
-  void acquireRecoveryMemory(KeyExtent extent) {
-    if (!extent.isMeta()) {
-      recoveryLock.lock();
-    }
-  }
+  private static final AutoCloseable NOOP_CLOSEABLE = () -> {};
 
-  void releaseRecoveryMemory(KeyExtent extent) {
-    if (!extent.isMeta()) {
-      recoveryLock.unlock();
+  AutoCloseable acquireRecoveryMemory(TabletMetadata tabletMetadata) {
+    if (tabletMetadata.getExtent().isMeta() || tabletMetadata.getLogs().isEmpty()) {
+      return NOOP_CLOSEABLE;
+    } else {
+      recoveryLock.lock();
+      return recoveryLock::unlock;
     }
   }
 
@@ -610,41 +594,14 @@
     thriftClientHandler = newTabletClientHandler(watcher, writeTracker);
     scanClientHandler = newThriftScanClientHandler(writeTracker);
 
-    TProcessor processor = ThriftProcessorTypes.getTabletServerTProcessor(clientHandler,
-        thriftClientHandler, scanClientHandler, getContext());
+    TProcessor processor =
+        ThriftProcessorTypes.getTabletServerTProcessor(clientHandler, thriftClientHandler,
+            scanClientHandler, thriftClientHandler, thriftClientHandler, getContext());
     HostAndPort address = startServer(getConfiguration(), clientAddress.getHost(), processor);
     log.info("address = {}", address);
     return address;
   }
 
-  @Deprecated
-  private void startReplicationService() throws UnknownHostException {
-    final var handler =
-        new org.apache.accumulo.tserver.replication.ReplicationServicerHandler(this);
-    var processor = ThriftProcessorTypes.getReplicationClientTProcessor(handler, getContext());
-    Property maxMessageSizeProperty =
-        getConfiguration().get(Property.TSERV_MAX_MESSAGE_SIZE) != null
-            ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE;
-    ServerAddress sp = TServerUtils.startServer(getContext(), clientAddress.getHost(),
-        Property.REPLICATION_RECEIPT_SERVICE_PORT, processor, "ReplicationServicerHandler",
-        "Replication Servicer", Property.TSERV_PORTSEARCH, Property.REPLICATION_MIN_THREADS, null,
-        Property.REPLICATION_THREADCHECK, maxMessageSizeProperty);
-    this.replServer = sp.server;
-    log.info("Started replication service on {}", sp.address);
-
-    try {
-      // The replication service is unique to the thrift service for a tserver, not just a host.
-      // Advertise the host and port for replication service given the host and port for the
-      // tserver.
-      getContext().getZooReaderWriter().putPersistentData(getContext().getZooKeeperRoot()
-          + org.apache.accumulo.core.replication.ReplicationConstants.ZOO_TSERVERS + "/"
-          + clientAddress, sp.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
-    } catch (Exception e) {
-      log.error("Could not advertise replication service port", e);
-      throw new RuntimeException(e);
-    }
-  }
-
   @Override
   public ServiceLock getLock() {
     return tabletServerLock;
@@ -655,11 +612,6 @@
     return managerLockCache;
   }
 
-  @Override
-  public GarbageCollectionLogger getGcLogger() {
-    return gcLogger;
-  }
-
   private void announceExistence() {
     ZooReaderWriter zoo = getContext().getZooReaderWriter();
     try {
@@ -676,7 +628,8 @@
         throw e;
       }
 
-      tabletServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, UUID.randomUUID());
+      UUID tabletServerUUID = UUID.randomUUID();
+      tabletServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, tabletServerUUID);
 
       LockWatcher lw = new LockWatcher() {
 
@@ -686,7 +639,7 @@
             if (!serverStopRequested) {
               log.error("Lost tablet server lock (reason = {}), exiting.", reason);
             }
-            gcLogger.logGCInfo(getConfiguration());
+            context.getLowMemoryDetector().logGCInfo(getConfiguration());
           });
         }
 
@@ -697,12 +650,18 @@
         }
       };
 
-      byte[] lockContent = new ServerServices(getClientAddressString(), Service.TSERV_CLIENT)
-          .toString().getBytes(UTF_8);
       for (int i = 0; i < 120 / 5; i++) {
         zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
 
-        if (tabletServerLock.tryLock(lw, lockContent)) {
+        ServiceDescriptors descriptors = new ServiceDescriptors();
+        for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT,
+            ThriftService.TABLET_INGEST, ThriftService.TABLET_MANAGEMENT, ThriftService.TABLET_SCAN,
+            ThriftService.TSERV}) {
+          descriptors
+              .addService(new ServiceDescriptor(tabletServerUUID, svc, getClientAddressString()));
+        }
+
+        if (tabletServerLock.tryLock(lw, new ServiceLockData(descriptors))) {
           lockID = tabletServerLock.getLockID()
               .serialize(getContext().getZooKeeperRoot() + Constants.ZTSERVERS + "/");
           lockSessionId = tabletServerLock.getSessionId();
@@ -722,25 +681,11 @@
     }
   }
 
-  @Deprecated
-  private void initializeZkForReplication() {
-    try {
-      org.apache.accumulo.server.replication.ZooKeeperInitialization.ensureZooKeeperInitialized(
-          getContext().getZooReaderWriter(), getContext().getZooKeeperRoot());
-    } catch (KeeperException | InterruptedException e) {
-      throw new IllegalStateException("Exception while ensuring ZooKeeper is initialized", e);
-    }
-  }
-
   // main loop listens for client requests
   @Override
   public void run() {
     SecurityUtil.serverLogin(getConfiguration());
 
-    // To make things easier on users/devs, and to avoid creating an upgrade path to 1.7
-    // We can just make the zookeeper paths before we try to use.
-    initializeZkForReplication();
-
     if (authKeyWatcher != null) {
       log.info("Seeding ZooKeeper watcher for authentication keys");
       try {
@@ -768,11 +713,12 @@
     scanMetrics = new TabletServerScanMetrics();
     mincMetrics = new TabletServerMinCMetrics();
     ceMetrics = new CompactionExecutorsMetrics();
+    pausedMetrics = new PausedCompactionMetrics();
     blockCacheMetrics = new BlockCacheMetrics(this.resourceManager.getIndexCache(),
         this.resourceManager.getDataCache(), this.resourceManager.getSummaryCache());
 
-    metricsInfo.addMetricsProducers(metrics, updateMetrics, scanMetrics, mincMetrics, ceMetrics,
-        blockCacheMetrics);
+    metricsInfo.addMetricsProducers(this, metrics, updateMetrics, scanMetrics, mincMetrics,
+        ceMetrics, pausedMetrics, blockCacheMetrics);
     metricsInfo.init();
 
     this.compactionManager = new CompactionManager(() -> Iterators
@@ -789,21 +735,6 @@
       throw new RuntimeException(e);
     }
 
-    @SuppressWarnings("deprecation")
-    ThreadPoolExecutor distWorkQThreadPool = ThreadPools.getServerThreadPools()
-        .createExecutorService(getConfiguration(), Property.TSERV_WORKQ_THREADS, true);
-
-    // TODO: Remove when Property.TSERV_WORKQ_THREADS is removed
-    DistributedWorkQueue bulkFailedCopyQ =
-        new DistributedWorkQueue(getContext().getZooKeeperRoot() + Constants.ZBULK_FAILED_COPYQ,
-            getConfiguration(), getContext());
-    try {
-      bulkFailedCopyQ.startProcessing(new BulkFailedCopyProcessor(getContext()),
-          distWorkQThreadPool);
-    } catch (Exception e1) {
-      throw new RuntimeException("Failed to start distributed work queue for copying ", e1);
-    }
-
     try {
       logSorter.startWatchingForRecoveryLogs();
     } catch (Exception ex) {
@@ -811,18 +742,6 @@
       throw new RuntimeException(ex);
     }
     final AccumuloConfiguration aconf = getConfiguration();
-    // if the replication name is ever set, then start replication services
-    @SuppressWarnings("deprecation")
-    Property p = Property.REPLICATION_NAME;
-    ScheduledFuture<?> future = context.getScheduledExecutor().scheduleWithFixedDelay(() -> {
-      if (this.replServer == null) {
-        if (!getConfiguration().get(p).isEmpty()) {
-          log.info(p.getKey() + " was set, starting repl services.");
-          setupReplication(aconf);
-        }
-      }
-    }, 0, 5, TimeUnit.SECONDS);
-    watchNonCriticalScheduledTask(future);
 
     long tabletCheckFrequency = aconf.getTimeInMillis(Property.TSERV_HEALTH_CHECK_FREQ);
     // Periodically check that metadata of tablets matches what is held in memory
@@ -840,10 +759,11 @@
       Duration duration;
       Span mdScanSpan = TraceUtil.startSpan(this.getClass(), "metadataScan");
       try (Scope scope = mdScanSpan.makeCurrent()) {
+        List<KeyExtent> missingTablets = new ArrayList<>();
         // gather metadata for all tablets readTablets()
-        try (TabletsMetadata tabletsMetadata =
-            getContext().getAmple().readTablets().forTablets(onlineTabletsSnapshot.keySet())
-                .fetch(FILES, LOGS, ECOMP, PREV_ROW).build()) {
+        try (TabletsMetadata tabletsMetadata = getContext().getAmple().readTablets()
+            .forTablets(onlineTabletsSnapshot.keySet(), Optional.of(missingTablets::add))
+            .fetch(FILES, LOGS, ECOMP, PREV_ROW).build()) {
           duration = Duration.between(start, Instant.now());
           log.debug("Metadata scan took {}ms for {} tablets read.", duration.toMillis(),
               onlineTabletsSnapshot.keySet().size());
@@ -855,6 +775,13 @@
             MetadataUpdateCount counter = updateCounts.get(extent);
             tablet.compareTabletInfo(counter, tabletMetadata);
           }
+
+          for (var extent : missingTablets) {
+            Tablet tablet = onlineTabletsSnapshot.get(extent);
+            if (!tablet.isClosed()) {
+              log.error("Tablet {} is open but does not exist in metadata table.", extent);
+            }
+          }
         }
       } catch (Exception e) {
         log.error("Unable to complete verification of tablet metadata", e);
@@ -945,10 +872,6 @@
         }
       }
     }
-    log.debug("Stopping Replication Server");
-    if (this.replServer != null) {
-      this.replServer.stop();
-    }
 
     log.debug("Stopping Thrift Servers");
     if (server != null) {
@@ -962,7 +885,7 @@
       log.warn("Failed to close filesystem : {}", e.getMessage(), e);
     }
 
-    gcLogger.logGCInfo(getConfiguration());
+    context.getLowMemoryDetector().logGCInfo(getConfiguration());
 
     log.info("TServerInfo: stop requested. exiting ... ");
 
@@ -973,30 +896,6 @@
     }
   }
 
-  @SuppressWarnings("deprecation")
-  private void setupReplication(AccumuloConfiguration aconf) {
-    // Start the thrift service listening for incoming replication requests
-    try {
-      startReplicationService();
-    } catch (UnknownHostException e) {
-      throw new RuntimeException("Failed to start replication service", e);
-    }
-
-    // Start the pool to handle outgoing replications
-    final ThreadPoolExecutor replicationThreadPool = ThreadPools.getServerThreadPools()
-        .createExecutorService(getConfiguration(), Property.REPLICATION_WORKER_THREADS);
-    replWorker.setExecutor(replicationThreadPool);
-    replWorker.run();
-
-    // Check the configuration value for the size of the pool and, if changed, resize the pool
-    Runnable replicationWorkThreadPoolResizer = () -> {
-      ThreadPools.resizePool(replicationThreadPool, aconf, Property.REPLICATION_WORKER_THREADS);
-    };
-    ScheduledFuture<?> future = context.getScheduledExecutor()
-        .scheduleWithFixedDelay(replicationWorkThreadPoolResizer, 10, 30, TimeUnit.SECONDS);
-    watchNonCriticalScheduledTask(future);
-  }
-
   public String getClientAddressString() {
     if (clientAddress == null) {
       return null;
@@ -1055,21 +954,6 @@
     Threads.createThread("Split/MajC initiator", new MajorCompactor(context)).start();
 
     clientAddress = HostAndPort.fromParts(getHostname(), 0);
-
-    final AccumuloConfiguration aconf = getConfiguration();
-
-    @SuppressWarnings("removal")
-    Property TSERV_MONITOR_FS = Property.TSERV_MONITOR_FS;
-    if (aconf.getBoolean(TSERV_MONITOR_FS)) {
-      log.warn("{} is deprecated and marked for removal.", TSERV_MONITOR_FS.getKey());
-      FileSystemMonitor.start(aconf);
-    }
-
-    Runnable gcDebugTask = () -> gcLogger.logGCInfo(getConfiguration());
-
-    ScheduledFuture<?> future = context.getScheduledExecutor().scheduleWithFixedDelay(gcDebugTask,
-        0, TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
-    watchNonCriticalScheduledTask(future);
   }
 
   public TabletServerStatus getStats(Map<TableId,MapCounter<ScanRunState>> scanCounts) {
@@ -1172,9 +1056,9 @@
   private Durability getMincEventDurability(KeyExtent extent) {
     TableConfiguration conf;
     if (extent.isMeta()) {
-      conf = getContext().getTableConfiguration(RootTable.ID);
+      conf = getContext().getTableConfiguration(AccumuloTable.ROOT.tableId());
     } else {
-      conf = getContext().getTableConfiguration(MetadataTable.ID);
+      conf = getContext().getTableConfiguration(AccumuloTable.METADATA.tableId());
     }
     return DurabilityImpl.fromString(conf.get(Property.TABLE_DURABILITY));
   }
@@ -1187,19 +1071,17 @@
   }
 
   public void minorCompactionStarted(CommitSession tablet, long lastUpdateSequence,
-      String newMapfileLocation) throws IOException {
+      String newDataFileLocation) throws IOException {
     Durability durability = getMincEventDurability(tablet.getExtent());
-    logger.minorCompactionStarted(tablet, lastUpdateSequence, newMapfileLocation, durability);
+    logger.minorCompactionStarted(tablet, lastUpdateSequence, newDataFileLocation, durability);
   }
 
   public void recover(VolumeManager fs, KeyExtent extent, List<LogEntry> logEntries,
       Set<String> tabletFiles, MutationReceiver mutationReceiver) throws IOException {
     List<Path> recoveryDirs = new ArrayList<>();
-    List<LogEntry> sorted = new ArrayList<>(logEntries);
-    sorted.sort((e1, e2) -> (int) (e1.timestamp - e2.timestamp));
-    for (LogEntry entry : sorted) {
+    for (LogEntry entry : logEntries) {
       Path recovery = null;
-      Path finished = RecoveryPath.getRecoveryPath(new Path(entry.filename));
+      Path finished = RecoveryPath.getRecoveryPath(new Path(entry.getPath()));
       finished = SortedLogState.getFinishedMarkerPath(finished);
       TabletServer.log.debug("Looking for " + finished);
       if (fs.exists(finished)) {
@@ -1227,21 +1109,6 @@
     return getContext().getTableConfiguration(extent.tableId());
   }
 
-  public DfsLogger.ServerResources getServerConfig() {
-    return new DfsLogger.ServerResources() {
-
-      @Override
-      public VolumeManager getVolumeManager() {
-        return TabletServer.this.getVolumeManager();
-      }
-
-      @Override
-      public AccumuloConfiguration getConfiguration() {
-        return TabletServer.this.getConfiguration();
-      }
-    };
-  }
-
   public SortedMap<KeyExtent,Tablet> getOnlineTablets() {
     return onlineTablets.snapshot();
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index c3ad1fd..2bb8030 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.tserver;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toUnmodifiableMap;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -81,10 +81,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.Suppliers;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -262,8 +262,8 @@
 
     try {
       cacheManager = BlockCacheManagerFactory.getInstance(acuConf);
-    } catch (Exception e) {
-      throw new RuntimeException("Error creating BlockCacheManager", e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalStateException("Error creating BlockCacheManager", e);
     }
 
     cacheManager.start(tserver.getBlockCacheConfiguration(acuConf));
@@ -373,7 +373,7 @@
     int maxOpenFiles = acuConf.getCount(Property.TSERV_SCAN_MAX_OPENFILES);
 
     fileLenCache =
-        CacheBuilder.newBuilder().maximumSize(Math.min(maxOpenFiles * 1000L, 100_000)).build();
+        Caffeine.newBuilder().maximumSize(Math.min(maxOpenFiles * 1000L, 100_000)).build();
 
     fileManager = new FileManager(context, maxOpenFiles, fileLenCache);
 
@@ -665,9 +665,9 @@
       return tableConf;
     }
 
-    // BEGIN methods that Tablets call to manage their set of open map files
+    // BEGIN methods that Tablets call to manage their set of open data files
 
-    public void importedMapFiles() {
+    public void importedDataFiles() {
       lastReportedCommitTime = System.currentTimeMillis();
     }
 
@@ -680,7 +680,7 @@
           new ScanCacheProvider(tableConf, scanDispatch, _iCache, _dCache));
     }
 
-    // END methods that Tablets call to manage their set of open map files
+    // END methods that Tablets call to manage their set of open data files
 
     // BEGIN methods that Tablets call to manage memory
 
@@ -767,12 +767,6 @@
     }
   }
 
-  @SuppressWarnings("deprecation")
-  private static abstract class DispatchParamsImpl implements DispatchParameters,
-      org.apache.accumulo.core.spi.scan.ScanDispatcher.DispatchParmaters {
-
-  }
-
   public void executeReadAhead(KeyExtent tablet, ScanDispatcher dispatcher, ScanSession scanInfo,
       Runnable task) {
 
@@ -787,7 +781,7 @@
       scanInfo.scanParams.setScanDispatch(ScanDispatch.builder().build());
       scanExecutors.get("meta").execute(task);
     } else {
-      DispatchParameters params = new DispatchParamsImpl() {
+      DispatchParameters params = new DispatchParameters() {
 
         // in scan critical path so only create ServiceEnv if needed
         private final Supplier<ServiceEnvironment> senvSupplier =
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
index 8a99b23..a89af66 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.tserver;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -38,6 +38,7 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.TabletType;
 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.conf.Property;
 import org.apache.accumulo.core.data.Column;
@@ -59,14 +60,13 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
-import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 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.TSamplerConfiguration;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
-import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.TooManyFilesException;
 import org.apache.accumulo.server.rpc.TServerUtils;
@@ -133,8 +133,8 @@
       boolean isolated, long readaheadThreshold, TSamplerConfiguration tSamplerConfig,
       long batchTimeOut, String contextArg, Map<String,String> executionHints, long busyTimeout)
       throws NotServingTabletException, ThriftSecurityException,
-      org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException,
-      TSampleNotPresentException, ScanServerBusyException {
+      org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException, TSampleNotPresentException,
+      ScanServerBusyException {
     final KeyExtent extent = KeyExtent.fromThrift(textent);
     TabletResolver resolver = new TabletResolver() {
       @Override
@@ -156,7 +156,7 @@
       boolean isolated, long readaheadThreshold, TSamplerConfiguration tSamplerConfig,
       long batchTimeOut, String contextArg, Map<String,String> executionHints,
       ScanSession.TabletResolver tabletResolver, long busyTimeout) throws NotServingTabletException,
-      ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException,
+      ThriftSecurityException, org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException,
       TSampleNotPresentException, ScanServerBusyException {
 
     server.getScanMetrics().incrementStartScan(1.0D);
@@ -230,8 +230,8 @@
   @Override
   public ScanResult continueScan(TInfo tinfo, long scanID, long busyTimeout)
       throws NoSuchScanIDException, NotServingTabletException,
-      org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException,
-      TSampleNotPresentException, ScanServerBusyException {
+      org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException, TSampleNotPresentException,
+      ScanServerBusyException {
     SingleScanSession scanSession =
         (SingleScanSession) server.getSessionManager().reserveSession(scanID);
     if (scanSession == null) {
@@ -247,8 +247,8 @@
 
   protected ScanResult continueScan(TInfo tinfo, long scanID, SingleScanSession scanSession,
       long busyTimeout) throws NoSuchScanIDException, NotServingTabletException,
-      org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException,
-      TSampleNotPresentException, ScanServerBusyException {
+      org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException, TSampleNotPresentException,
+      ScanServerBusyException {
 
     server.getScanMetrics().incrementContinueScan(1.0D);
 
@@ -268,7 +268,7 @@
       if (e.getCause() instanceof NotServingTabletException) {
         throw (NotServingTabletException) e.getCause();
       } else if (e.getCause() instanceof TooManyFilesException) {
-        throw new org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException(
+        throw new org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException(
             scanSession.extent.toThrift());
       } else if (e.getCause() instanceof SampleNotPresentException) {
         throw new TSampleNotPresentException(scanSession.extent.toThrift());
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
index d307f72..d904a2e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
@@ -22,6 +22,7 @@
 import java.util.Collections;
 
 import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.data.constraints.Constraint;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.security.AuthorizationContainer;
@@ -30,9 +31,7 @@
 import org.apache.accumulo.server.constraints.SystemEnvironment;
 import org.apache.accumulo.server.security.SecurityOperation;
 
-@SuppressWarnings("deprecation")
-public class TservConstraintEnv
-    implements SystemEnvironment, org.apache.accumulo.core.constraints.Constraint.Environment {
+public class TservConstraintEnv implements SystemEnvironment, Constraint.Environment {
 
   private final ServerContext context;
   private final TCredentials credentials;
@@ -50,11 +49,6 @@
   }
 
   @Override
-  public KeyExtent getExtent() {
-    return ke;
-  }
-
-  @Override
   public TabletId getTablet() {
     return new TabletIdImpl(ke);
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
index ee94180..27c1048 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.tserver;
 
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
-
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.manager.thrift.TabletLoadState;
@@ -27,7 +25,9 @@
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.TabletLocationState.BadLocationStateException;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
+import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
+import org.apache.accumulo.core.util.time.NanoTime;
+import org.apache.accumulo.core.util.time.SteadyTime;
 import org.apache.accumulo.server.manager.state.DistributedStoreException;
 import org.apache.accumulo.server.manager.state.TabletStateStore;
 import org.apache.accumulo.tserver.managermessage.TabletStatusMessage;
@@ -39,15 +39,17 @@
   private static final Logger log = LoggerFactory.getLogger(UnloadTabletHandler.class);
   private final KeyExtent extent;
   private final TUnloadTabletGoal goalState;
-  private final long requestTimeSkew;
+  private final SteadyTime requestTime;
+  private final NanoTime createTime;
   private final TabletServer server;
 
   public UnloadTabletHandler(TabletServer server, KeyExtent extent, TUnloadTabletGoal goalState,
-      long requestTime) {
+      SteadyTime requestTime) {
     this.extent = extent;
     this.goalState = goalState;
     this.server = server;
-    this.requestTimeSkew = requestTime - NANOSECONDS.toMillis(System.nanoTime());
+    this.requestTime = requestTime;
+    this.createTime = NanoTime.now();
   }
 
   @Override
@@ -111,8 +113,7 @@
       TServerInstance instance = server.getTabletSession();
       TabletLocationState tls = null;
       try {
-        tls = new TabletLocationState(extent, null, Location.current(instance), null, null, null,
-            false);
+        tls = new TabletLocationState(extent, null, Location.current(instance), null, null, null);
       } catch (BadLocationStateException e) {
         log.error("Unexpected error", e);
       }
@@ -122,7 +123,7 @@
         TabletStateStore.unassign(server.getContext(), tls, null);
       } else {
         TabletStateStore.suspend(server.getContext(), tls, null,
-            requestTimeSkew + NANOSECONDS.toMillis(System.nanoTime()));
+            requestTime.plus(createTime.elapsed()));
       }
     } catch (DistributedStoreException ex) {
       log.warn("Unable to update storage", ex);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionPlan.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionPlan.java
deleted file mode 100644
index ec801b1..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionPlan.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.tserver.compaction;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-
-import com.google.common.collect.Sets;
-
-/**
- * A plan for a compaction: the input files, the files that are *not* inputs to a compaction that
- * should simply be deleted, and the optional parameters used to create the resulting output file.
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class CompactionPlan {
-  public final List<StoredTabletFile> inputFiles = new ArrayList<>();
-  public final List<StoredTabletFile> deleteFiles = new ArrayList<>();
-  public WriteParameters writeParameters = null;
-
-  @Override
-  public String toString() {
-    StringBuilder b = new StringBuilder();
-    b.append(inputFiles);
-    if (!deleteFiles.isEmpty()) {
-      b.append(" files to be deleted ");
-      b.append(deleteFiles);
-      if (writeParameters != null) {
-        if (writeParameters.getCompressType() != null) {
-          b.append(" compress type " + writeParameters.getCompressType());
-        }
-        if (writeParameters.getHdfsBlockSize() != 0) {
-          b.append(" hdfs block size " + writeParameters.getHdfsBlockSize());
-        }
-        if (writeParameters.getBlockSize() != 0) {
-          b.append(" data block size " + writeParameters.getBlockSize());
-        }
-        if (writeParameters.getIndexBlockSize() != 0) {
-          b.append(" index block size " + writeParameters.getIndexBlockSize());
-        }
-        if (writeParameters.getReplication() != 0) {
-          b.append(" replication " + writeParameters.getReplication());
-        }
-      }
-    }
-    return b.toString();
-  }
-
-  /**
-   * Validate compaction plan.
-   *
-   * @param allFiles All possible files
-   * @throws IllegalStateException thrown when validation fails.
-   */
-  public final void validate(Set<StoredTabletFile> allFiles) {
-    Set<TabletFile> inputSet = new HashSet<>(inputFiles);
-    Set<StoredTabletFile> deleteSet = new HashSet<>(deleteFiles);
-
-    if (!allFiles.containsAll(inputSet)) {
-      inputSet.removeAll(allFiles);
-      throw new IllegalStateException("plan inputs contains files not in allFiles " + inputSet);
-    }
-
-    if (!allFiles.containsAll(deleteSet)) {
-      deleteSet.removeAll(allFiles);
-      throw new IllegalStateException("plan deletes contains files not in allFiles " + deleteSet);
-    }
-
-    if (!Collections.disjoint(inputSet, deleteSet)) {
-      throw new IllegalStateException("plan contains overlap in inputFiles and deleteFiles "
-          + Sets.intersection(inputSet, deleteSet));
-    }
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.java
deleted file mode 100644
index 09eb70d..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/CompactionStrategy.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.tserver.compaction;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
-import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
-import org.apache.accumulo.core.spi.compaction.CompactionPlanner;
-
-/**
- * The interface for customizing major compactions.
- * <p>
- * Important: Compaction configurations should be migrated to use only the new compaction plugins as
- * soon as possible. Interactions between this deprecated plugin with the new plugins may provide
- * suboptimal and/or inconsistent results when both are configured. For example, concurrent
- * compactions may not occur if a compaction strategy is in place along with the new compaction
- * plugins. This legacy compaction plugin has been maintained to support migrations to the new model
- * and will be removed in the next major release.
- * <p>
- * The tablet server has one thread to ask many tablets if they should compact. When the strategy
- * returns true, then tablet is added to the queue of tablets waiting for a compaction thread. Once
- * a thread is available, the {@link #gatherInformation(MajorCompactionRequest)} method is called
- * outside the tablets' lock. This gives the strategy the ability to read information that maybe
- * expensive to fetch. Once the gatherInformation returns, the tablet lock is grabbed and the
- * compactionPlan computed. This should *not* do expensive operations, especially not I/O. Note that
- * the number of files may change between calls to
- * {@link #gatherInformation(MajorCompactionRequest)} and
- * {@link #getCompactionPlan(MajorCompactionRequest)}.
- * <p>
- * <b>Note:</b> the strategy object used for the {@link #shouldCompact(MajorCompactionRequest)} call
- * is going to be different from the one used in the compaction thread.
- *
- * @deprecated since 2.1.0 use {@link CompactionSelector}, {@link CompactionConfigurer}, and
- *             {@link CompactionPlanner} instead. See
- *             {@link org.apache.accumulo.core.client.admin.CompactionStrategyConfig} for more
- *             information about why this was deprecated.
- * @see org.apache.accumulo.core.spi.compaction
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public abstract class CompactionStrategy {
-  /**
-   * The settings for the compaction strategy pulled from zookeeper. The
-   * <code>table.compacations.major.strategy.opts</code> part of the setting will be removed.
-   */
-  public void init(Map<String,String> options) {}
-
-  /**
-   * Determine if this tablet is eligible for a major compaction. It's ok if it later determines
-   * (through {@link #gatherInformation(MajorCompactionRequest)} and
-   * {@link #getCompactionPlan(MajorCompactionRequest)}) that it does not need to. Any state stored
-   * during shouldCompact will no longer exist when
-   * {@link #gatherInformation(MajorCompactionRequest)} and
-   * {@link #getCompactionPlan(MajorCompactionRequest)} are called.
-   *
-   * <p>
-   * Called while holding the tablet lock, so it should not be doing any blocking.
-   *
-   * <p>
-   * Since no blocking should be done in this method, then its unexpected that this method will
-   * throw IOException. However, since it is in the API, it can not be easily removed.
-   */
-  public abstract boolean shouldCompact(MajorCompactionRequest request) throws IOException;
-
-  /**
-   * Called prior to obtaining the tablet lock, useful for examining metadata or indexes. State
-   * collected during this method will be available during the call the
-   * {@link #getCompactionPlan(MajorCompactionRequest)}.
-   *
-   * @param request basic details about the tablet
-   */
-  public void gatherInformation(MajorCompactionRequest request) throws IOException {}
-
-  /**
-   * Get the plan for compacting a tablets files. Called while holding the tablet lock, so it should
-   * not be doing any blocking.
-   *
-   * <p>
-   * Since no blocking should be done in this method, then its unexpected that this method will
-   * throw IOException. However, since it is in the API, it can not be easily removed.
-   *
-   * @param request basic details about the tablet
-   * @return the plan for a major compaction, or null to cancel the compaction.
-   */
-  public abstract CompactionPlan getCompactionPlan(MajorCompactionRequest request)
-      throws IOException;
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategy.java
deleted file mode 100644
index 6fcff74..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategy.java
+++ /dev/null
@@ -1,243 +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.tserver.compaction;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-@SuppressWarnings("removal")
-public class DefaultCompactionStrategy extends CompactionStrategy {
-
-  /**
-   * Keeps track of the sum of the size of all files within a window. The files are sorted from
-   * largest to smallest. Supports efficiently creating sub windows, sliding the window, and
-   * shrinking the window.
-   */
-  @VisibleForTesting
-  static class SizeWindow {
-
-    List<CompactionFile> files;
-    long sum = 0;
-
-    int first;
-    int last;
-
-    SizeWindow() {}
-
-    SizeWindow(Map<StoredTabletFile,DataFileValue> allFiles) {
-      files = new ArrayList<>();
-      for (Entry<StoredTabletFile,DataFileValue> entry : allFiles.entrySet()) {
-        files.add(new CompactionFile(entry.getKey(), entry.getValue().getSize()));
-      }
-
-      files.sort(Comparator.comparingLong(CompactionFile::getSize)
-          .thenComparing(CompactionFile::getFile).reversed());
-
-      for (CompactionFile file : files) {
-        sum += file.size;
-      }
-
-      first = 0;
-      last = files.size();
-    }
-
-    void pop() {
-      if (first >= last) {
-        throw new IllegalStateException("Can not pop");
-      }
-
-      sum -= files.get(first).size;
-      first++;
-    }
-
-    long topSize() {
-      return files.get(first).size;
-    }
-
-    boolean slideUp() {
-      if (first == 0) {
-        return false;
-      }
-
-      first--;
-      last--;
-
-      sum += files.get(first).size;
-      sum -= files.get(last).size;
-
-      return true;
-    }
-
-    SizeWindow tail(int windowSize) {
-      Preconditions.checkArgument(windowSize > 0);
-
-      SizeWindow sub = new SizeWindow();
-
-      sub.files = files;
-      sub.first = Math.max(last - windowSize, first);
-      sub.last = last;
-      sub.sum = 0;
-
-      for (int i = sub.first; i < sub.last; i++) {
-        sub.sum += files.get(i).size;
-      }
-
-      return sub;
-    }
-
-    long sum() {
-      return sum;
-    }
-
-    int size() {
-      return (last - first);
-    }
-
-    public List<StoredTabletFile> getFiles() {
-      List<StoredTabletFile> windowFiles = new ArrayList<>(size());
-      for (int i = first; i < last; i++) {
-        windowFiles.add(files.get(i).file);
-      }
-      return windowFiles;
-    }
-
-    @Override
-    public String toString() {
-      return "size:" + size() + " sum:" + sum() + " first:" + first + " last:" + last + " topSize:"
-          + topSize();
-    }
-  }
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-    CompactionPlan plan = getCompactionPlan(request);
-    return plan != null && !plan.inputFiles.isEmpty();
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-    CompactionPlan result = new CompactionPlan();
-
-    List<StoredTabletFile> toCompact = findMapFilesToCompact(request);
-    if (toCompact == null || toCompact.isEmpty()) {
-      return result;
-    }
-    result.inputFiles.addAll(toCompact);
-    return result;
-  }
-
-  private static class CompactionFile {
-    public StoredTabletFile file;
-    public long size;
-
-    public CompactionFile(StoredTabletFile file, long size) {
-      this.file = file;
-      this.size = size;
-    }
-
-    long getSize() {
-      return size;
-    }
-
-    TabletFile getFile() {
-      return file;
-    }
-  }
-
-  private List<StoredTabletFile> findMapFilesToCompact(MajorCompactionRequest request) {
-    MajorCompactionReason reason = request.getReason();
-    if (reason == MajorCompactionReason.USER) {
-      return new ArrayList<>(request.getFiles().keySet());
-    }
-
-    if (reason == MajorCompactionReason.CHOP) {
-      // should not happen, but this is safe
-      return new ArrayList<>(request.getFiles().keySet());
-    }
-
-    if (request.getFiles().size() <= 1) {
-      return null;
-    }
-
-    double ratio = Double.parseDouble(request.getTableConfig(Property.TABLE_MAJC_RATIO.getKey()));
-    int maxFilesToCompact =
-        Integer.parseInt(request.getTableConfig(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()));
-    int maxFilesPerTablet = request.getMaxFilesPerTablet();
-
-    int minFilesToCompact = 0;
-    if (request.getFiles().size() > maxFilesPerTablet) {
-      minFilesToCompact = request.getFiles().size() - maxFilesPerTablet + 1;
-    }
-
-    minFilesToCompact = Math.min(minFilesToCompact, maxFilesToCompact);
-
-    SizeWindow all = new SizeWindow(request.getFiles());
-
-    List<StoredTabletFile> files = null;
-
-    // Within a window of size maxFilesToCompact containing the smallest files check to see if any
-    // files meet the compaction ratio criteria.
-    SizeWindow window = all.tail(maxFilesToCompact);
-    while (window.size() > 1 && files == null) {
-
-      if (window.topSize() * ratio <= window.sum()) {
-        files = window.getFiles();
-      }
-
-      window.pop();
-    }
-
-    // Previous search was fruitless. If there are more files than maxFilesToCompact, then try
-    // sliding the window up looking for files that meet the criteria.
-    if (files == null || files.size() < minFilesToCompact) {
-      window = all.tail(maxFilesToCompact);
-
-      files = null;
-
-      // When moving the window up there is no need to pop/shrink the window. All possible sets are
-      // covered without doing this. Proof is left as an exercise for the reader. This is predicated
-      // on the first search shrinking the initial window.
-      while (window.slideUp() && files == null) {
-        if (window.topSize() * ratio <= window.sum()) {
-          files = window.getFiles();
-        }
-      }
-    }
-
-    // Ensure the minimum number of files are compacted.
-    if ((files != null && files.size() < minFilesToCompact)
-        || (files == null && minFilesToCompact > 0)) {
-      // get the smallest files of size minFilesToCompact
-      files = all.tail(minFilesToCompact).getFiles();
-    }
-
-    return files;
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/EverythingCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/EverythingCompactionStrategy.java
deleted file mode 100644
index 656acad..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/EverythingCompactionStrategy.java
+++ /dev/null
@@ -1,42 +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.tserver.compaction;
-
-/**
- * The default compaction strategy for user initiated compactions. This strategy will always select
- * all files.
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class EverythingCompactionStrategy extends CompactionStrategy {
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-    return true; // ACCUMULO-3645 compact for empty files too
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-    CompactionPlan plan = new CompactionPlan();
-    plan.inputFiles.addAll(request.getFiles().keySet());
-    return plan;
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionReason.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionReason.java
deleted file mode 100644
index 0ad595d..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionReason.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.tserver.compaction;
-
-@Deprecated(since = "2.1.0", forRemoval = true)
-public enum MajorCompactionReason {
-  // do not change the order, the order of this enum determines the order
-  // in which queued major compactions are executed
-  USER, CHOP, NORMAL, IDLE
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
deleted file mode 100644
index 41153d6..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.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.tserver.compaction;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.function.Predicate;
-
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.rfile.RFile.WriterOptions;
-import org.apache.accumulo.core.client.summary.Summarizer;
-import org.apache.accumulo.core.client.summary.Summarizer.Combiner;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.client.summary.Summary;
-import org.apache.accumulo.core.client.summary.Summary.FileStatistics;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-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.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.spi.cache.BlockCache;
-import org.apache.accumulo.core.summary.Gatherer;
-import org.apache.accumulo.core.summary.SummarizerFactory;
-import org.apache.accumulo.core.summary.SummaryCollection;
-import org.apache.accumulo.core.summary.SummaryReader;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactionStrategy;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.cache.Cache;
-
-/**
- * Information that can be used to determine how a tablet is to be major compacted, if needed.
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class MajorCompactionRequest implements Cloneable {
-  private final KeyExtent extent;
-  private final MajorCompactionReason reason;
-  private final VolumeManager volumeManager;
-  private final AccumuloConfiguration tableConfig;
-  private final BlockCache indexCache;
-  private final BlockCache summaryCache;
-  private Map<StoredTabletFile,DataFileValue> files;
-  private final ServerContext context;
-  private final Cache<String,Long> fileLenCache;
-
-  public MajorCompactionRequest(KeyExtent extent, MajorCompactionReason reason,
-      VolumeManager manager, AccumuloConfiguration tabletConfig, BlockCache summaryCache,
-      BlockCache indexCache, Cache<String,Long> fileLenCache, ServerContext context) {
-    this.extent = extent;
-    this.reason = reason;
-    this.volumeManager = manager;
-    this.tableConfig = tabletConfig;
-    this.files = Collections.emptyMap();
-    this.summaryCache = summaryCache;
-    this.indexCache = indexCache;
-    this.fileLenCache = fileLenCache;
-    this.context = context;
-  }
-
-  public MajorCompactionRequest(KeyExtent extent, MajorCompactionReason reason,
-      AccumuloConfiguration tabletConfig, ServerContext context) {
-    this(extent, reason, null, tabletConfig, null, null, null, context);
-  }
-
-  public MajorCompactionRequest(MajorCompactionRequest mcr) {
-    this(mcr.extent, mcr.reason, mcr.volumeManager, mcr.tableConfig, mcr.summaryCache,
-        mcr.indexCache, mcr.fileLenCache, mcr.context);
-    // know this is already unmodifiable, no need to wrap again
-    this.files = mcr.files;
-  }
-
-  @VisibleForTesting
-  public TabletId getTabletId() {
-    return new TabletIdImpl(extent);
-  }
-
-  public MajorCompactionReason getReason() {
-    return reason;
-  }
-
-  public Map<StoredTabletFile,DataFileValue> getFiles() {
-    return files;
-  }
-
-  /**
-   * Returns all summaries present in each file.
-   *
-   * <p>
-   * This method can only be called from
-   * {@link CompactionStrategy#gatherInformation(MajorCompactionRequest)}. Unfortunately,
-   * {@code gatherInformation()} is not called before
-   * {@link CompactionStrategy#shouldCompact(MajorCompactionRequest)}. Therefore
-   * {@code shouldCompact()} should just return true when a compactions strategy wants to use
-   * summary information.
-   *
-   * <p>
-   * When using summaries to make compaction decisions, it's important to ensure that all summary
-   * data fits in the tablet server summary cache. The size of this cache is configured by code
-   * tserver.cache.summary.size}. Also it's important to use the summarySelector predicate to only
-   * retrieve the needed summary data. Otherwise unneeded summary data could be brought into the
-   * cache.
-   *
-   * <p>
-   * Some files may contain data outside of a tablets range. When {@link Summarizer}'s generate
-   * small amounts of summary data, multiple summaries may be stored within a file for different row
-   * ranges. This will allow more accurate summaries to be returned for the case where a file has
-   * data outside a tablets range. However, some summary data outside of the tablets range may still
-   * be included. When this happens {@link FileStatistics#getExtra()} will be non zero. Also, its
-   * good to be aware of the other potential causes of inaccuracies
-   * {@link FileStatistics#getInaccurate()}
-   *
-   * <p>
-   * When this method is called with multiple files, it will automatically merge summary data using
-   * {@link Combiner#merge(Map, Map)}. If summary information is needed for each file, then just
-   * call this method for each file.
-   *
-   * <p>
-   * Writing a compaction strategy that uses summary information is a bit tricky. See the source
-   * code for {@link TooManyDeletesCompactionStrategy} as an example of a compaction strategy.
-   *
-   * @see Summarizer
-   * @see TableOperations#addSummarizers(String, SummarizerConfiguration...)
-   * @see WriterOptions#withSummarizers(SummarizerConfiguration...)
-   */
-  public List<Summary> getSummaries(Collection<StoredTabletFile> files,
-      Predicate<SummarizerConfiguration> summarySelector) {
-    Objects.requireNonNull(volumeManager,
-        "Getting summaries is not  supported at this time. It's only supported when "
-            + "CompactionStrategy.gatherInformation() is called.");
-    SummaryCollection sc = new SummaryCollection();
-    SummarizerFactory factory = new SummarizerFactory(tableConfig);
-    for (TabletFile file : files) {
-      FileSystem fs = volumeManager.getFileSystemByPath(file.getPath());
-      Configuration conf = context.getHadoopConf();
-      SummaryCollection fsc = SummaryReader
-          .load(fs, conf, factory, file.getPath(), summarySelector, summaryCache, indexCache,
-              fileLenCache, context.getTableConfiguration(extent.tableId()).getCryptoService())
-          .getSummaries(Collections.singletonList(new Gatherer.RowRange(extent)));
-      sc.merge(fsc, factory);
-    }
-
-    return sc.getSummaries();
-  }
-
-  public void setFiles(Map<StoredTabletFile,DataFileValue> update) {
-    this.files = Collections.unmodifiableMap(update);
-  }
-
-  public Map<String,String> getTableProperties() {
-    return tableConfig.getAllPropertiesWithPrefix(Property.TABLE_PREFIX);
-  }
-
-  public String getTableConfig(String key) {
-    Property property = Property.getPropertyByKey(key);
-    if (property == null || property.isSensitive()) {
-      throw new RuntimeException("Unable to access the configuration value " + key);
-    }
-    return tableConfig.get(property);
-  }
-
-  public int getMaxFilesPerTablet() {
-    return tableConfig.getMaxFilesPerTablet();
-  }
-
-  @Override
-  public MajorCompactionRequest clone() throws CloneNotSupportedException {
-    return (MajorCompactionRequest) super.clone();
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategy.java
deleted file mode 100644
index b8f9fb6..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategy.java
+++ /dev/null
@@ -1,77 +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.tserver.compaction;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.tserver.compaction.strategies.BasicCompactionStrategy;
-
-/**
- * {@link BasicCompactionStrategy} offer the same functionality as this class and more.
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class SizeLimitCompactionStrategy extends DefaultCompactionStrategy {
-  public static final String SIZE_LIMIT_OPT = "sizeLimit";
-
-  private long limit;
-
-  @Override
-  public void init(Map<String,String> options) {
-    limit = ConfigurationTypeHelper.getFixedMemoryAsBytes(options.get(SIZE_LIMIT_OPT));
-  }
-
-  private MajorCompactionRequest filterFiles(MajorCompactionRequest mcr) {
-    Map<StoredTabletFile,DataFileValue> filteredFiles = new HashMap<>();
-    for (Entry<StoredTabletFile,DataFileValue> entry : mcr.getFiles().entrySet()) {
-      if (entry.getValue().getSize() <= limit) {
-        filteredFiles.put(entry.getKey(), entry.getValue());
-      }
-    }
-
-    mcr = new MajorCompactionRequest(mcr);
-    mcr.setFiles(filteredFiles);
-
-    return mcr;
-  }
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-    return super.shouldCompact(filterFiles(request));
-  }
-
-  @Override
-  public void gatherInformation(MajorCompactionRequest request) throws IOException {
-    super.gatherInformation(filterFiles(request));
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-    return super.getCompactionPlan(filterFiles(request));
-  }
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/WriteParameters.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/WriteParameters.java
deleted file mode 100644
index 2acc152..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/WriteParameters.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.tserver.compaction;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class WriteParameters {
-  private String compressType = null;
-  private long hdfsBlockSize = 0;
-  private long blockSize = 0;
-  private long indexBlockSize = 0;
-  private int replication = 0;
-
-  public String getCompressType() {
-    return compressType;
-  }
-
-  public void setCompressType(String compressType) {
-    this.compressType = compressType;
-  }
-
-  public long getHdfsBlockSize() {
-    return hdfsBlockSize;
-  }
-
-  public void setHdfsBlockSize(long hdfsBlockSize) {
-    checkArgument(hdfsBlockSize >= 0);
-    this.hdfsBlockSize = hdfsBlockSize;
-  }
-
-  public long getBlockSize() {
-    return blockSize;
-  }
-
-  public void setBlockSize(long blockSize) {
-    checkArgument(blockSize >= 0);
-    this.blockSize = blockSize;
-  }
-
-  public long getIndexBlockSize() {
-    return indexBlockSize;
-  }
-
-  public void setIndexBlockSize(long indexBlockSize) {
-    checkArgument(indexBlockSize >= 0);
-    this.indexBlockSize = indexBlockSize;
-  }
-
-  public int getReplication() {
-    return replication;
-  }
-
-  public void setReplication(int replication) {
-    checkArgument(replication >= 0);
-    this.replication = replication;
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/BasicCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/BasicCompactionStrategy.java
deleted file mode 100644
index 8ce037a..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/BasicCompactionStrategy.java
+++ /dev/null
@@ -1,178 +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.tserver.compaction.strategies;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.admin.compaction.CompressionConfigurer;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-import org.apache.accumulo.tserver.compaction.WriteParameters;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A compaction strategy that covers the following uses cases.
- *
- * <ul>
- * <li>Filtering out input files larger than a specified size. These are never considered for
- * compaction.
- * <li>Compressing output files differently when the sum of the input files exceeds a specified
- * size.
- * </ul>
- *
- * <p>
- * To filter out input files based on size set
- * {@code table.majc.compaction.strategy.opts.filter.size} to the desired size.
- *
- * <p>
- * To use a different compression for larger inputs set
- * {@code table.majc.compaction.strategy.opts.large.compress.threshold } to bytes and
- * {@code  table.majc.compaction.strategy.opts.large.compress.type} to a compression type like gz or
- * snappy. When setting one of these properties then the other must be set. When the total size of
- * files being compacted is larger than the threshold then the specified compression type is used.
- *
- * <p>
- * To use this strategy with Minor Compactions set {@code table.file.compress.type=snappy} and set a
- * different compress type in {@code table.majc.compaction.strategy.opts.large.compress.type} for
- * larger files.
- *
- * <p>
- * The options that take sizes are in bytes and the suffixes K,M,and G can be used.
- *
- * @deprecated since 2.1.0 see {@link CompressionConfigurer}. Also compaction planners introduced in
- *             2.1.0 have the ability to avoid compacting files over a certain size. See
- *             {@link DefaultCompactionPlanner}
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class BasicCompactionStrategy extends DefaultCompactionStrategy {
-
-  private static final Logger log = LoggerFactory.getLogger(BasicCompactionStrategy.class);
-
-  public static final String SIZE_LIMIT_OPT = "filter.size";
-
-  /**
-   * Threshold memory in bytes. Files larger than this threshold will use
-   * <code>table.majc.compaction.strategy.opts.file.large.compress.type</code> for compression
-   */
-  public static final String LARGE_FILE_COMPRESSION_THRESHOLD = "large.compress.threshold";
-
-  /**
-   * Type of compression to use if large threshold is surpassed. One of "none", "gz", "bzip2",
-   * "lzo", "lz4", "snappy", or "zstd"
-   */
-  public static final String LARGE_FILE_COMPRESSION_TYPE = "large.compress.type";
-
-  private Long filterSize;
-  private Long largeThresh;
-  private String largeCompress;
-
-  @Override
-  public void init(Map<String,String> options) {
-    String limitVal = options.get(SIZE_LIMIT_OPT);
-    if (limitVal != null) {
-      filterSize = ConfigurationTypeHelper.getFixedMemoryAsBytes(limitVal);
-    }
-
-    String largeThresh = options.get(LARGE_FILE_COMPRESSION_THRESHOLD);
-    String largeCompress = options.get(LARGE_FILE_COMPRESSION_TYPE);
-    if (largeThresh != null && largeCompress != null) {
-      this.largeThresh = ConfigurationTypeHelper.getFixedMemoryAsBytes(largeThresh);
-      this.largeCompress = largeCompress;
-    } else if (largeThresh != null ^ largeCompress != null) {
-      throw new IllegalArgumentException("Must set both of "
-          + Property.TABLE_COMPACTION_STRATEGY_PREFIX + " (" + LARGE_FILE_COMPRESSION_TYPE + " and "
-          + LARGE_FILE_COMPRESSION_THRESHOLD + ") or neither for " + this.getClass().getName());
-    }
-
-  }
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-    return super.shouldCompact(filterFiles(request));
-  }
-
-  @Override
-  public void gatherInformation(MajorCompactionRequest request) throws IOException {
-    super.gatherInformation(filterFiles(request));
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-
-    request = filterFiles(request);
-
-    CompactionPlan plan = super.getCompactionPlan(request);
-
-    if (largeThresh != null) {
-
-      Long totalSize = calculateTotalSize(request, plan);
-
-      if (totalSize > largeThresh) {
-        plan.writeParameters = new WriteParameters();
-        if (log.isDebugEnabled()) {
-          log.debug("Changed compressType to {}: totalSize({}) was greater than threshold {}",
-              largeCompress, totalSize, largeThresh);
-        }
-        plan.writeParameters.setCompressType(largeCompress);
-      }
-    }
-
-    return plan;
-
-  }
-
-  private MajorCompactionRequest filterFiles(MajorCompactionRequest mcr) {
-    if (filterSize != null) {
-      Map<StoredTabletFile,DataFileValue> filteredFiles = new HashMap<>();
-      mcr.getFiles().forEach((fr, dfv) -> {
-        if (dfv.getSize() <= filterSize) {
-          filteredFiles.put(fr, dfv);
-        }
-      });
-
-      mcr = new MajorCompactionRequest(mcr);
-      mcr.setFiles(filteredFiles);
-    }
-    return mcr;
-  }
-
-  /**
-   * Calculates the total size of input files in the compaction plan
-   */
-  private Long calculateTotalSize(MajorCompactionRequest request, CompactionPlan plan) {
-    long totalSize = 0;
-    Map<StoredTabletFile,DataFileValue> allFiles = request.getFiles();
-    for (StoredTabletFile fileRef : plan.inputFiles) {
-      totalSize += allFiles.get(fileRef).getSize();
-    }
-    return totalSize;
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
deleted file mode 100644
index fda95a3..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/strategies/TooManyDeletesCompactionStrategy.java
+++ /dev/null
@@ -1,195 +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.tserver.compaction.strategies;
-
-import static org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer.DELETES_STAT;
-import static org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer.TOTAL_STAT;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.function.Predicate;
-
-import org.apache.accumulo.core.client.admin.compaction.TooManyDeletesSelector;
-import org.apache.accumulo.core.client.rfile.RFile.WriterOptions;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.client.summary.Summary;
-import org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-
-/**
- * This compaction strategy works in concert with the {@link DeletesSummarizer}. Using the
- * statistics from DeleteSummarizer this strategy will compact all files in a table when the number
- * of deletes/non-deletes exceeds a threshold.
- *
- * <p>
- * This strategy has two options. First the {@value #THRESHOLD_OPT} option allows setting the point
- * at which a compaction will be triggered. This options defaults to {@value #THRESHOLD_OPT_DEFAULT}
- * and must be in the range (0.0, 1.0]. The second option is {@value #PROCEED_ZERO_NO_SUMMARY_OPT}
- * which determines if the strategy should proceed when a bulk imported file has no summary
- * information.
- *
- * <p>
- * If the delete summarizer was configured on a table that already had files, then those files will
- * have not summary information. This strategy can still proceed in this situation. It will fall
- * back to using Accumulo's estimated entries per file in this case. For the files without summary
- * information the estimated number of deletes will be zero. This fall back method will
- * underestimate deletes which will not lead to false positives, except for the case of bulk
- * imported files. Accumulo estimates that bulk imported files have zero entires. The second option
- * {@value #PROCEED_ZERO_NO_SUMMARY_OPT} determines if this strategy should proceed when it sees
- * bulk imported files that do not have summary data. This option defaults to
- * {@value #PROCEED_ZERO_NO_SUMMARY_OPT_DEFAULT}.
- *
- * <p>
- * Bulk files can be generated with summary information by calling
- * {@code AccumuloFileOutputFormat#setSummarizers(JobConf, SummarizerConfiguration...)} or
- * {@link WriterOptions#withSummarizers(SummarizerConfiguration...)}
- *
- * <p>
- * When this strategy does not decide to compact based on the number of deletes, then it will defer
- * the decision to the {@link DefaultCompactionStrategy}.
- *
- * <p>
- * Configuring this compaction strategy for a table will cause it to always queue compactions, even
- * though it may not decide to compact. These queued compactions may show up on the Accumulo monitor
- * page. This is because summary data can not be read until after compaction is queued and dequeued.
- * When the compaction is dequeued it can then decide not to compact. See <a
- * href=https://issues.apache.org/jira/browse/ACCUMULO-4573>ACCUMULO-4573</a>
- *
- * @since 2.0.0
- * @deprecated since 2.1.0 use {@link TooManyDeletesSelector} instead
- */
-// Eclipse might show @SuppressWarnings("removal") as unnecessary.
-// Eclipse is wrong. See https://bugs.eclipse.org/bugs/show_bug.cgi?id=565271
-@SuppressWarnings("removal")
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class TooManyDeletesCompactionStrategy extends DefaultCompactionStrategy {
-
-  private boolean shouldCompact = false;
-
-  private double threshold;
-
-  private boolean proceed_bns;
-
-  /**
-   * This option should be a floating point number between 1 and 0.
-   */
-  public static final String THRESHOLD_OPT = "threshold";
-
-  /**
-   * The default threshold.
-   */
-  public static final String THRESHOLD_OPT_DEFAULT = ".25";
-
-  public static final String PROCEED_ZERO_NO_SUMMARY_OPT = "proceed_zero_no_summary";
-
-  public static final String PROCEED_ZERO_NO_SUMMARY_OPT_DEFAULT = "false";
-
-  @Override
-  public void init(Map<String,String> options) {
-    this.threshold = Double.parseDouble(options.getOrDefault(THRESHOLD_OPT, THRESHOLD_OPT_DEFAULT));
-    if (threshold <= 0.0 || threshold > 1.0) {
-      throw new IllegalArgumentException(
-          "Threshold must be in range (0.0, 1.0], saw : " + threshold);
-    }
-
-    this.proceed_bns = Boolean.parseBoolean(
-        options.getOrDefault(PROCEED_ZERO_NO_SUMMARY_OPT, PROCEED_ZERO_NO_SUMMARY_OPT_DEFAULT));
-  }
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-    Collection<SummarizerConfiguration> configuredSummarizers =
-        SummarizerConfiguration.fromTableProperties(request.getTableProperties());
-
-    // check if delete summarizer is configured for table
-    if (configuredSummarizers.stream().map(SummarizerConfiguration::getClassName)
-        .anyMatch(cn -> cn.equals(DeletesSummarizer.class.getName()))) {
-      // This is called before gatherInformation, so need to always queue for compaction until
-      // context
-      // can be gathered. Also its not safe to request summary
-      // information here as its a blocking operation. Blocking operations are not allowed in
-      // shouldCompact.
-      return true;
-    } else {
-      return super.shouldCompact(request);
-    }
-  }
-
-  @Override
-  public void gatherInformation(MajorCompactionRequest request) throws IOException {
-    super.gatherInformation(request);
-
-    Predicate<SummarizerConfiguration> summarizerPredicate =
-        conf -> conf.getClassName().equals(DeletesSummarizer.class.getName())
-            && conf.getOptions().isEmpty();
-
-    long total = 0;
-    long deletes = 0;
-
-    for (Entry<StoredTabletFile,DataFileValue> entry : request.getFiles().entrySet()) {
-      Collection<Summary> summaries =
-          request.getSummaries(Collections.singleton(entry.getKey()), summarizerPredicate);
-      if (summaries.size() == 1) {
-        Summary summary = summaries.iterator().next();
-        total += summary.getStatistics().get(TOTAL_STAT);
-        deletes += summary.getStatistics().get(DELETES_STAT);
-      } else {
-        long numEntries = entry.getValue().getNumEntries();
-        if (numEntries == 0 && !proceed_bns) {
-          shouldCompact = false;
-          return;
-        } else {
-          // no summary data so use Accumulo's estimate of total entries in file
-          total += entry.getValue().getNumEntries();
-        }
-      }
-    }
-
-    long nonDeletes = total - deletes;
-
-    if (nonDeletes >= 0) {
-      // check nonDeletes >= 0 because if this is not true then its clear evidence that the
-      // estimates are off
-
-      double ratio = deletes / (double) nonDeletes;
-      shouldCompact = ratio >= threshold;
-    } else {
-      shouldCompact = false;
-    }
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-    if (shouldCompact) {
-      CompactionPlan cp = new CompactionPlan();
-      cp.inputFiles.addAll(request.getFiles().keySet());
-      return cp;
-    }
-
-    // fall back to default
-    return super.getCompactionPlan(request);
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionManager.java
index 4a244c4..cd88329 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionManager.java
@@ -22,6 +22,7 @@
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 
+import java.time.Duration;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -54,9 +55,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 CompactionManager {
@@ -78,8 +79,6 @@
 
   private CompactionExecutorsMetrics ceMetrics;
 
-  private String lastDeprecationWarning = "";
-
   private Map<CompactionExecutorId,ExternalCompactionExecutor> externalExecutors;
 
   private Map<ExternalCompactionId,ExtCompInfo> runningExternalCompactions;
@@ -97,21 +96,14 @@
     }
   }
 
-  private void warnAboutDeprecation(String warning) {
-    if (!warning.equals(lastDeprecationWarning)) {
-      log.warn(warning);
-      lastDeprecationWarning = warning;
-    }
-  }
-
   private void mainLoop() {
     long lastCheckAllTime = System.nanoTime();
 
     long increment = Math.max(1, maxTimeBetweenChecks / 10);
 
-    var retryFactory = Retry.builder().infiniteRetries().retryAfter(increment, MILLISECONDS)
-        .incrementBy(increment, MILLISECONDS).maxWait(maxTimeBetweenChecks, MILLISECONDS)
-        .backOffFactor(1.07).logInterval(1, MINUTES).createFactory();
+    var retryFactory = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(increment))
+        .incrementBy(Duration.ofMillis(increment)).maxWait(Duration.ofMillis(maxTimeBetweenChecks))
+        .backOffFactor(1.07).logInterval(Duration.ofMinutes(1)).createFactory();
     var retry = retryFactory.createRetry();
     Compactable last = null;
 
@@ -196,8 +188,7 @@
       CompactionExecutorsMetrics ceMetrics) {
     this.compactables = compactables;
 
-    this.currentCfg =
-        new CompactionServicesConfig(context.getConfiguration(), this::warnAboutDeprecation);
+    this.currentCfg = new CompactionServicesConfig(context.getConfiguration());
 
     this.context = context;
 
@@ -209,14 +200,13 @@
 
     Map<CompactionServiceId,CompactionService> tmpServices = new HashMap<>();
 
-    unknownCompactionServiceErrorCache =
-        CacheBuilder.newBuilder().expireAfterWrite(5, MINUTES).build();
+    unknownCompactionServiceErrorCache = Caffeine.newBuilder().expireAfterWrite(5, MINUTES).build();
 
     currentCfg.getPlanners().forEach((serviceName, plannerClassName) -> {
       try {
         tmpServices.put(CompactionServiceId.of(serviceName),
             new CompactionService(serviceName, plannerClassName,
-                currentCfg.getRateLimit(serviceName),
+                currentCfg.getPlannerPrefix(serviceName), currentCfg.getRateLimit(serviceName),
                 currentCfg.getOptions().getOrDefault(serviceName, Map.of()), context, ceMetrics,
                 this::getExternalExecutor));
       } catch (RuntimeException e) {
@@ -247,8 +237,7 @@
 
       lastConfigCheckTime = System.nanoTime();
 
-      var tmpCfg =
-          new CompactionServicesConfig(context.getConfiguration(), this::warnAboutDeprecation);
+      var tmpCfg = new CompactionServicesConfig(context.getConfiguration());
 
       if (!currentCfg.equals(tmpCfg)) {
         Map<CompactionServiceId,CompactionService> tmpServices = new HashMap<>();
@@ -261,11 +250,12 @@
             if (service == null) {
               tmpServices.put(csid,
                   new CompactionService(serviceName, plannerClassName,
-                      tmpCfg.getRateLimit(serviceName),
+                      tmpCfg.getPlannerPrefix(serviceName), tmpCfg.getRateLimit(serviceName),
                       tmpCfg.getOptions().getOrDefault(serviceName, Map.of()), context, ceMetrics,
                       this::getExternalExecutor));
             } else {
-              service.configurationChanged(plannerClassName, tmpCfg.getRateLimit(serviceName),
+              service.configurationChanged(plannerClassName, tmpCfg.getPlannerPrefix(serviceName),
+                  tmpCfg.getRateLimit(serviceName),
                   tmpCfg.getOptions().getOrDefault(serviceName, Map.of()));
               tmpServices.put(csid, service);
             }
@@ -396,6 +386,7 @@
     public CompactionExecutorId ceid;
     public int running;
     public int queued;
+    public int paused;
   }
 
   public Collection<ExtCompMetric> getExternalMetrics() {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
index 3054db1..bb74044 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionService.java
@@ -67,9 +67,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 CompactionService {
@@ -94,8 +94,8 @@
 
   private static final Logger log = LoggerFactory.getLogger(CompactionService.class);
 
-  public CompactionService(String serviceName, String plannerClass, Long maxRate,
-      Map<String,String> plannerOptions, ServerContext context,
+  public CompactionService(String serviceName, String plannerClass, String plannerPrefix,
+      Long maxRate, Map<String,String> plannerOptions, ServerContext context,
       CompactionExecutorsMetrics ceMetrics,
       Function<CompactionExecutorId,ExternalCompactionExecutor> externExecutorSupplier) {
 
@@ -108,8 +108,8 @@
     this.ceMetrics = ceMetrics;
     this.externExecutorSupplier = externExecutorSupplier;
 
-    var initParams =
-        new CompactionPlannerInitParams(myId, plannerOpts, new ServiceEnvironmentImpl(context));
+    var initParams = new CompactionPlannerInitParams(myId, plannerPrefix, plannerOpts,
+        new ServiceEnvironmentImpl(context));
     planner = createPlanner(myId, plannerClass, plannerOptions, initParams);
 
     Map<CompactionExecutorId,CompactionExecutor> tmpExecutors = new HashMap<>();
@@ -126,7 +126,7 @@
           new InternalCompactionExecutor(ceid, numThreads, ceMetrics, readLimiter, writeLimiter));
     });
 
-    initParams.getRequestedExternalExecutors().forEach((ceid) -> {
+    initParams.getRequestedExternalExecutors().forEach(ceid -> {
       tmpExecutors.put(ceid, externExecutorSupplier.apply(ceid));
     });
 
@@ -140,7 +140,7 @@
       queuedForPlanning.put(kind, new ConcurrentHashMap<KeyExtent,Compactable>());
     }
 
-    maxScanFilesExceededErrorCache = CacheBuilder.newBuilder().expireAfterWrite(5, MINUTES).build();
+    maxScanFilesExceededErrorCache = Caffeine.newBuilder().expireAfterWrite(5, MINUTES).build();
 
     log.debug("Created new compaction service id:{} rate limit:{} planner:{} planner options:{}",
         myId, maxRate, plannerClass, plannerOptions);
@@ -327,7 +327,8 @@
       Compactable compactable, Consumer<Compactable> completionCallback) {
     // log error if tablet is metadata and compaction is external
     var execIds = plan.getJobs().stream().map(cj -> (CompactionExecutorIdImpl) cj.getExecutor());
-    if (compactable.getExtent().isMeta() && execIds.anyMatch(ceid -> ceid.isExternalId())) {
+    if (compactable.getExtent().isMeta()
+        && execIds.anyMatch(CompactionExecutorIdImpl::isExternalId)) {
       log.error(
           "Compacting metadata tablets on external compactors is not supported, please change "
               + "config for compaction service ({}) and/or table ASAP.  {} is not compacting, "
@@ -390,7 +391,7 @@
         .anyMatch(job -> job.getStatus() == Status.QUEUED);
   }
 
-  public void configurationChanged(String plannerClassName, Long maxRate,
+  public void configurationChanged(String plannerClassName, String plannerPrefix, Long maxRate,
       Map<String,String> plannerOptions) {
     Preconditions.checkArgument(maxRate >= 0);
 
@@ -403,8 +404,8 @@
       return;
     }
 
-    var initParams =
-        new CompactionPlannerInitParams(myId, plannerOptions, new ServiceEnvironmentImpl(context));
+    var initParams = new CompactionPlannerInitParams(myId, plannerPrefix, plannerOptions,
+        new ServiceEnvironmentImpl(context));
     var tmpPlanner = createPlanner(myId, plannerClassName, plannerOptions, initParams);
 
     Map<CompactionExecutorId,CompactionExecutor> tmpExecutors = new HashMap<>();
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/ExternalCompactionJob.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/ExternalCompactionJob.java
index fcd044c..a06787d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/ExternalCompactionJob.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/ExternalCompactionJob.java
@@ -26,8 +26,8 @@
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.spi.compaction.CompactionKind;
@@ -41,7 +41,7 @@
 
   private Map<StoredTabletFile,DataFileValue> jobFiles;
   private boolean propagateDeletes;
-  private TabletFile compactTmpName;
+  private ReferencedTabletFile compactTmpName;
   private KeyExtent extent;
   private ExternalCompactionId externalCompactionId;
   private CompactionKind kind;
@@ -52,7 +52,7 @@
   public ExternalCompactionJob() {}
 
   public ExternalCompactionJob(Map<StoredTabletFile,DataFileValue> jobFiles,
-      boolean propagateDeletes, TabletFile compactTmpName, KeyExtent extent,
+      boolean propagateDeletes, ReferencedTabletFile compactTmpName, KeyExtent extent,
       ExternalCompactionId externalCompactionId, CompactionKind kind, List<IteratorSetting> iters,
       Long userCompactionId, Map<String,String> overrides) {
     this.jobFiles = Objects.requireNonNull(jobFiles);
@@ -76,12 +76,12 @@
 
     List<InputFile> files = jobFiles.entrySet().stream().map(e -> {
       var dfv = e.getValue();
-      return new InputFile(e.getKey().getPathStr(), dfv.getSize(), dfv.getNumEntries(),
+      return new InputFile(e.getKey().getMetadata(), dfv.getSize(), dfv.getNumEntries(),
           dfv.getTime());
     }).collect(Collectors.toList());
 
     return new TExternalCompactionJob(externalCompactionId.toString(), extent.toThrift(), files,
-        iteratorSettings, compactTmpName.getPathStr(), propagateDeletes,
+        iteratorSettings, compactTmpName.getNormalizedPathStr(), propagateDeletes,
         org.apache.accumulo.core.tabletserver.thrift.TCompactionKind.valueOf(kind.name()),
         userCompactionId, overrides);
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 1f56568..020144f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -20,6 +20,8 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Collections.singletonList;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_FINISH;
 import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_START;
 import static org.apache.accumulo.tserver.logger.LogEvents.DEFINE_TABLET;
@@ -56,6 +58,8 @@
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.spi.crypto.FileDecrypter;
 import org.apache.accumulo.core.spi.crypto.FileEncrypter;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.ServerContext;
@@ -73,14 +77,13 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 
 /**
  * Wrap a connection to a logger.
  *
  */
-public class DfsLogger implements Comparable<DfsLogger> {
+public final class DfsLogger implements Comparable<DfsLogger> {
   // older version supported for upgrade
   public static final String LOG_FILE_HEADER_V3 = "--- Log File Header (v3) ---";
 
@@ -115,18 +118,11 @@
     }
   }
 
-  public interface ServerResources {
-    AccumuloConfiguration getConfiguration();
-
-    VolumeManager getVolumeManager();
-  }
-
-  private final LinkedBlockingQueue<DfsLogger.LogWork> workQueue = new LinkedBlockingQueue<>();
+  private final LinkedBlockingQueue<LogWork> workQueue = new LinkedBlockingQueue<>();
 
   private final Object closeLock = new Object();
 
-  private static final DfsLogger.LogWork CLOSED_MARKER =
-      new DfsLogger.LogWork(null, Durability.FLUSH);
+  private static final LogWork CLOSED_MARKER = new LogWork(null, Durability.FLUSH);
 
   private static final LogFileValue EMPTY = new LogFileValue();
 
@@ -135,9 +131,19 @@
   private class LogSyncingTask implements Runnable {
     private int expectedReplication = 0;
 
+    private final AtomicLong syncCounter;
+    private final AtomicLong flushCounter;
+    private final long slowFlushMillis;
+
+    LogSyncingTask(AtomicLong syncCounter, AtomicLong flushCounter, long slowFlushMillis) {
+      this.syncCounter = syncCounter;
+      this.flushCounter = flushCounter;
+      this.slowFlushMillis = slowFlushMillis;
+    }
+
     @Override
     public void run() {
-      ArrayList<DfsLogger.LogWork> work = new ArrayList<>();
+      ArrayList<LogWork> work = new ArrayList<>();
       boolean sawClosedMarker = false;
       while (!sawClosedMarker) {
         work.clear();
@@ -168,7 +174,7 @@
           }
         }
 
-        long start = System.currentTimeMillis();
+        long start = System.nanoTime();
         try {
           if (shouldHSync.isPresent()) {
             if (shouldHSync.orElseThrow()) {
@@ -182,11 +188,10 @@
         } catch (IOException | RuntimeException ex) {
           fail(work, ex, "synching");
         }
-        long duration = System.currentTimeMillis() - start;
-        if (duration > slowFlushMillis) {
-          String msg = new StringBuilder(128).append("Slow sync cost: ").append(duration)
-              .append(" ms, current pipeline: ").append(Arrays.toString(getPipeLine())).toString();
-          log.info(msg);
+        long duration = System.nanoTime() - start;
+        if (duration > MILLISECONDS.toNanos(slowFlushMillis)) {
+          log.info("Slow sync cost: {} ms, current pipeline: {}", NANOSECONDS.toMillis(duration),
+              Arrays.toString(getPipeLine()));
           if (expectedReplication > 0) {
             int current = expectedReplication;
             try {
@@ -211,7 +216,7 @@
           }
         }
 
-        for (DfsLogger.LogWork logWork : work) {
+        for (LogWork logWork : work) {
           if (logWork == CLOSED_MARKER) {
             sawClosedMarker = true;
           } else {
@@ -221,9 +226,9 @@
       }
     }
 
-    private void fail(ArrayList<DfsLogger.LogWork> work, Exception ex, String why) {
+    private void fail(ArrayList<LogWork> work, Exception ex, String why) {
       log.warn("Exception {} {}", why, ex, ex);
-      for (DfsLogger.LogWork logWork : work) {
+      for (LogWork logWork : work) {
         logWork.exception = ex;
       }
     }
@@ -280,59 +285,61 @@
 
   @Override
   public boolean equals(Object obj) {
-    // filename is unique
     if (obj == null) {
       return false;
     }
     if (obj instanceof DfsLogger) {
-      return getFileName().equals(((DfsLogger) obj).getFileName());
+      return logEntry.equals(((DfsLogger) obj).logEntry);
     }
     return false;
   }
 
   @Override
   public int hashCode() {
-    // filename is unique
-    return getFileName().hashCode();
+    return logEntry.hashCode();
   }
 
-  private final ServerContext context;
-  private final ServerResources conf;
   private FSDataOutputStream logFile;
   private DataOutputStream encryptingLogFile = null;
-  private String logPath;
+  private final LogEntry logEntry;
   private Thread syncThread;
 
-  /* Track what's actually in +r/!0 for this logger ref */
-  private String metaReference;
-  private AtomicLong syncCounter;
-  private AtomicLong flushCounter;
-  private final long slowFlushMillis;
   private long writes = 0;
 
-  private DfsLogger(ServerContext context, ServerResources conf) {
-    this.context = context;
-    this.conf = conf;
-    this.slowFlushMillis =
-        conf.getConfiguration().getTimeInMillis(Property.TSERV_SLOW_FLUSH_MILLIS);
-  }
+  /**
+   * Create a new DfsLogger with the provided characteristics.
+   */
+  public static DfsLogger createNew(ServerContext context, AtomicLong syncCounter,
+      AtomicLong flushCounter, String address) throws IOException {
 
-  public DfsLogger(ServerContext context, ServerResources conf, AtomicLong syncCounter,
-      AtomicLong flushCounter) {
-    this(context, conf);
-    this.syncCounter = syncCounter;
-    this.flushCounter = flushCounter;
+    String filename = UUID.randomUUID().toString();
+    String addressForFilename = address.replace(':', '+');
+
+    var chooserEnv =
+        new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.LOGGER, context);
+    String logPath =
+        context.getVolumeManager().choose(chooserEnv, context.getBaseUris()) + Path.SEPARATOR
+            + Constants.WAL_DIR + Path.SEPARATOR + addressForFilename + Path.SEPARATOR + filename;
+
+    LogEntry log = LogEntry.fromPath(logPath);
+    DfsLogger dfsLogger = new DfsLogger(log);
+    long slowFlushMillis =
+        context.getConfiguration().getTimeInMillis(Property.TSERV_SLOW_FLUSH_MILLIS);
+    dfsLogger.open(context, logPath, filename, address, syncCounter, flushCounter, slowFlushMillis);
+    return dfsLogger;
   }
 
   /**
    * Reference a pre-existing log file.
    *
-   * @param meta the cq for the "log" entry in +r/!0
+   * @param logEntry the "log" entry in +r/!0
    */
-  public DfsLogger(ServerContext context, ServerResources conf, String filename, String meta) {
-    this(context, conf);
-    this.logPath = filename;
-    metaReference = meta;
+  public static DfsLogger fromLogEntry(LogEntry logEntry) {
+    return new DfsLogger(logEntry);
+  }
+
+  private DfsLogger(LogEntry logEntry) {
+    this.logEntry = logEntry;
   }
 
   /**
@@ -349,11 +356,6 @@
     byte[] magic4 = DfsLogger.LOG_FILE_HEADER_V4.getBytes(UTF_8);
     byte[] magic3 = DfsLogger.LOG_FILE_HEADER_V3.getBytes(UTF_8);
 
-    if (magic4.length != magic3.length) {
-      throw new AssertionError("Always expect log file headers to be same length : " + magic4.length
-          + " != " + magic3.length);
-    }
-
     byte[] magicBuffer = new byte[magic4.length];
     try {
       input.readFully(magicBuffer);
@@ -395,29 +397,25 @@
    *
    * @param address The address of the host using this WAL
    */
-  public synchronized void open(String address) throws IOException {
-    String filename = UUID.randomUUID().toString();
+  private synchronized void open(ServerContext context, String logPath, String filename,
+      String address, AtomicLong syncCounter, AtomicLong flushCounter, long slowFlushMillis)
+      throws IOException {
     log.debug("Address is {}", address);
-    String logger = Joiner.on("+").join(address.split(":"));
 
     log.debug("DfsLogger.open() begin");
-    VolumeManager fs = conf.getVolumeManager();
 
-    var chooserEnv = new VolumeChooserEnvironmentImpl(
-        org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope.LOGGER, context);
-    logPath = fs.choose(chooserEnv, context.getBaseUris()) + Path.SEPARATOR + Constants.WAL_DIR
-        + Path.SEPARATOR + logger + Path.SEPARATOR + filename;
+    VolumeManager fs = context.getVolumeManager();
 
-    metaReference = toString();
-    LoggerOperation op = null;
+    LoggerOperation op;
+    var serverConf = context.getConfiguration();
     try {
       Path logfilePath = new Path(logPath);
-      short replication = (short) conf.getConfiguration().getCount(Property.TSERV_WAL_REPLICATION);
+      short replication = (short) serverConf.getCount(Property.TSERV_WAL_REPLICATION);
       if (replication == 0) {
         replication = fs.getDefaultReplication(logfilePath);
       }
-      long blockSize = getWalBlockSize(conf.getConfiguration());
-      if (conf.getConfiguration().getBoolean(Property.TSERV_WAL_SYNC)) {
+      long blockSize = getWalBlockSize(serverConf);
+      if (serverConf.getBoolean(Property.TSERV_WAL_SYNC)) {
         logFile = fs.createSyncable(logfilePath, 0, replication, blockSize);
       } else {
         logFile = fs.create(logfilePath, true, 0, replication, blockSize);
@@ -439,8 +437,8 @@
 
       // Initialize the log file with a header and its encryption
       CryptoEnvironment env = new CryptoEnvironmentImpl(Scope.WAL);
-      CryptoService cryptoService = context.getCryptoFactory().getService(env,
-          conf.getConfiguration().getAllCryptoProperties());
+      CryptoService cryptoService =
+          context.getCryptoFactory().getService(env, serverConf.getAllCryptoProperties());
       logFile.write(LOG_FILE_HEADER_V4.getBytes(UTF_8));
 
       log.debug("Using {} for encrypting WAL {}", cryptoService.getClass().getSimpleName(),
@@ -449,11 +447,11 @@
       byte[] cryptoParams = encrypter.getDecryptionParameters();
       CryptoUtils.writeParams(cryptoParams, logFile);
 
-      /**
-       * Always wrap the WAL in a NoFlushOutputStream to prevent extra flushing to HDFS. The
-       * {@link #write(LogFileKey, LogFileValue)} method will flush crypto data or do nothing when
-       * crypto is not enabled.
-       **/
+      /*
+       * Always wrap the WAL in a NoFlushOutputStream to prevent extra flushing to HDFS. The method
+       * write(LogFileKey, LogFileValue) will flush crypto data or do nothing when crypto is not
+       * enabled.
+       */
       OutputStream encryptedStream = encrypter.encryptStream(new NoFlushOutputStream(logFile));
       if (encryptedStream instanceof NoFlushOutputStream) {
         encryptingLogFile = (NoFlushOutputStream) encryptedStream;
@@ -475,7 +473,8 @@
       throw new IOException(ex);
     }
 
-    syncThread = Threads.createThread("Accumulo WALog thread " + this, new LogSyncingTask());
+    syncThread = Threads.createThread("Accumulo WALog thread " + this,
+        new LogSyncingTask(syncCounter, flushCounter, slowFlushMillis));
     syncThread.start();
     op.await();
     log.debug("Got new write-ahead log: {}", this);
@@ -484,38 +483,22 @@
   static long getWalBlockSize(AccumuloConfiguration conf) {
     long blockSize = conf.getAsBytes(Property.TSERV_WAL_BLOCKSIZE);
     if (blockSize == 0) {
-      @SuppressWarnings("deprecation")
-      Property prop = conf.resolve(Property.TSERV_WAL_MAX_SIZE, Property.TSERV_WALOG_MAX_SIZE);
-      blockSize = (long) (conf.getAsBytes(prop) * 1.1);
+      blockSize = (long) (conf.getAsBytes(Property.TSERV_WAL_MAX_SIZE) * 1.1);
     }
     return blockSize;
   }
 
   @Override
   public String toString() {
-    String fileName = getFileName();
-    if (fileName.contains(":")) {
-      return getLogger() + "/" + getFileName();
-    }
-    return fileName;
+    return logEntry.toString();
   }
 
-  /**
-   * get the cq needed to reference this logger's entry in +r/!0
-   */
-  public String getMeta() {
-    if (metaReference == null) {
-      throw new IllegalStateException("logger doesn't have meta reference. " + this);
-    }
-    return metaReference;
-  }
-
-  public String getFileName() {
-    return logPath;
+  public LogEntry getLogEntry() {
+    return logEntry;
   }
 
   public Path getPath() {
-    return new Path(logPath);
+    return new Path(logEntry.getPath());
   }
 
   public void close() throws IOException {
@@ -586,7 +569,7 @@
 
   private LoggerOperation logFileData(List<Pair<LogFileKey,LogFileValue>> keys,
       Durability durability) throws IOException {
-    DfsLogger.LogWork work = new DfsLogger.LogWork(new CountDownLatch(1), durability);
+    LogWork work = new LogWork(new CountDownLatch(1), durability);
     try {
       for (Pair<LogFileKey,LogFileValue> pair : keys) {
         write(pair.getFirst(), pair.getSecond());
@@ -672,14 +655,9 @@
     return logKeyData(key, durability);
   }
 
-  private String getLogger() {
-    String[] parts = logPath.split("/");
-    return Joiner.on(":").join(parts[parts.length - 2].split("[+]"));
-  }
-
   @Override
   public int compareTo(DfsLogger o) {
-    return getFileName().compareTo(o.getFileName());
+    return logEntry.getPath().compareTo(o.logEntry.getPath());
   }
 
   /*
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java
index 6fc396e..2b5a6c6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LogSorter.java
@@ -40,7 +40,8 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.master.thrift.RecoveryStatus;
+import org.apache.accumulo.core.manager.thrift.RecoveryStatus;
+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.Pair;
@@ -166,10 +167,7 @@
         return;
       }
 
-      @SuppressWarnings("deprecation")
-      Property prop = sortedLogConf.resolve(Property.TSERV_WAL_SORT_BUFFER_SIZE,
-          Property.TSERV_SORT_BUFFER_SIZE);
-      final long bufferSize = sortedLogConf.getAsBytes(prop);
+      final long bufferSize = sortedLogConf.getAsBytes(Property.TSERV_WAL_SORT_BUFFER_SIZE);
       Thread.currentThread().setName("Sorting " + name + " for recovery");
       while (true) {
         final ArrayList<Pair<LogFileKey,LogFileValue>> buffer = new ArrayList<>();
@@ -281,7 +279,7 @@
     }
 
     try (var writer = FileOperations.getInstance().newWriterBuilder()
-        .forFile(fullPath.toString(), fs, fs.getConf(), cryptoService)
+        .forFile(UnreferencedTabletFile.of(fs, fullPath), fs, fs.getConf(), cryptoService)
         .withTableConfiguration(sortedLogConf).build()) {
       writer.startDefaultLocalityGroup();
       for (var entry : keyListMap.entrySet()) {
@@ -293,9 +291,7 @@
   }
 
   public void startWatchingForRecoveryLogs() throws KeeperException, InterruptedException {
-    @SuppressWarnings("deprecation")
-    int threadPoolSize = this.conf.getCount(this.conf
-        .resolve(Property.TSERV_WAL_SORT_MAX_CONCURRENT, Property.TSERV_RECOVERY_MAX_CONCURRENT));
+    int threadPoolSize = this.conf.getCount(Property.TSERV_WAL_SORT_MAX_CONCURRENT);
     ThreadPoolExecutor threadPool =
         ThreadPools.getServerThreadPools().getPoolBuilder(this.getClass().getName())
             .numCoreThreads(threadPoolSize).enableThreadPoolMetrics().build();
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
index 469a128..f5b9736 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
@@ -36,6 +36,7 @@
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.iterators.IteratorAdapter;
+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.server.ServerContext;
@@ -80,7 +81,7 @@
 
     for (Path logDir : recoveryLogDirs) {
       LOG.debug("Opening recovery log dir {}", logDir.getName());
-      SortedSet<Path> logFiles = getFiles(vm, logDir);
+      SortedSet<UnreferencedTabletFile> logFiles = getFiles(vm, logDir);
       var fs = vm.getFileSystemByPath(logDir);
 
       // only check the first key once to prevent extra iterator creation and seeking
@@ -88,9 +89,9 @@
         validateFirstKey(context, cryptoService, fs, logFiles, logDir);
       }
 
-      for (Path log : logFiles) {
+      for (UnreferencedTabletFile log : logFiles) {
         FileSKVIterator fileIter = FileOperations.getInstance().newReaderBuilder()
-            .forFile(log.toString(), fs, fs.getConf(), cryptoService)
+            .forFile(log, fs, fs.getConf(), cryptoService)
             .withTableConfiguration(context.getConfiguration()).seekToBeginning().build();
         if (range != null) {
           fileIter.seek(range, Collections.emptySet(), false);
@@ -98,11 +99,13 @@
         Iterator<Entry<Key,Value>> scanIter = new IteratorAdapter(fileIter);
 
         if (scanIter.hasNext()) {
-          LOG.debug("Write ahead log {} has data in range {} {}", log.getName(), start, end);
+          LOG.debug("Write ahead log {} has data in range {} {}", log.getPath().getName(), start,
+              end);
           iterators.add(scanIter);
           fileIters.add(fileIter);
         } else {
-          LOG.debug("Write ahead log {} has no data in range {} {}", log.getName(), start, end);
+          LOG.debug("Write ahead log {} has no data in range {} {}", log.getPath().getName(), start,
+              end);
           fileIter.close();
         }
       }
@@ -137,12 +140,14 @@
   /**
    * Check for sorting signal files (finished/failed) and get the logs in the provided directory.
    */
-  private SortedSet<Path> getFiles(VolumeManager fs, Path directory) throws IOException {
+  private SortedSet<UnreferencedTabletFile> getFiles(VolumeManager fs, Path directory)
+      throws IOException {
     boolean foundFinish = false;
     // Path::getName compares the last component of each Path value. In this case, the last
     // component should
     // always have the format 'part-r-XXXXX.rf', where XXXXX are one-up values.
-    SortedSet<Path> logFiles = new TreeSet<>(Comparator.comparing(Path::getName));
+    SortedSet<UnreferencedTabletFile> logFiles =
+        new TreeSet<>(Comparator.comparing(tf -> tf.getPath().getName()));
     for (FileStatus child : fs.listStatus(directory)) {
       if (child.getPath().getName().startsWith("_")) {
         continue;
@@ -155,7 +160,8 @@
         continue;
       }
       FileSystem ns = fs.getFileSystemByPath(child.getPath());
-      Path fullLogPath = ns.makeQualified(child.getPath());
+      UnreferencedTabletFile fullLogPath =
+          UnreferencedTabletFile.of(ns, ns.makeQualified(child.getPath()));
       logFiles.add(fullLogPath);
     }
     if (!foundFinish) {
@@ -169,9 +175,9 @@
    * Check that the first entry in the WAL is OPEN. Only need to do this once.
    */
   private void validateFirstKey(ServerContext context, CryptoService cs, FileSystem fs,
-      SortedSet<Path> logFiles, Path fullLogPath) throws IOException {
+      SortedSet<UnreferencedTabletFile> logFiles, Path fullLogPath) throws IOException {
     try (FileSKVIterator fileIter = FileOperations.getInstance().newReaderBuilder()
-        .forFile(logFiles.first().toString(), fs, fs.getConf(), cs)
+        .forFile(logFiles.first(), fs, fs.getConf(), cs)
         .withTableConfiguration(context.getConfiguration()).seekToBeginning().build()) {
       Iterator<Entry<Key,Value>> iterator = new IteratorAdapter(fileIter);
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 346b701..54dad2b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -38,21 +38,19 @@
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.Halt;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.Retry.RetryFactory;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
-import org.apache.accumulo.tserver.log.DfsLogger.ServerResources;
 import org.apache.accumulo.tserver.tablet.CommitSession;
 import org.apache.hadoop.fs.Path;
+import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -184,17 +182,15 @@
   }
 
   /**
-   * Get the current WAL file
+   * Get the current log entry
    *
-   * @return The name of the current log, or null if there is no current log.
+   * @return the current log entry, or null if there is no current log
    */
-  public String getLogFile() {
+  @Nullable
+  public LogEntry getLogEntry() {
     logIdLock.readLock().lock();
     try {
-      if (currentLog == null) {
-        return null;
-      }
-      return currentLog.getFileName();
+      return currentLog == null ? null : currentLog.getLogEntry();
     } finally {
       logIdLock.readLock().unlock();
     }
@@ -218,7 +214,7 @@
       if (next instanceof DfsLogger) {
         currentLog = (DfsLogger) next;
         logId.incrementAndGet();
-        log.info("Using next log {}", currentLog.getFileName());
+        log.info("Using next log {}", currentLog.getLogEntry());
 
         // When we successfully create a WAL, make sure to reset the Retry.
         if (createRetry != null) {
@@ -226,7 +222,6 @@
         }
 
         this.createTime = System.currentTimeMillis();
-        return;
       } else {
         throw new RuntimeException("Error: unexpected type seen: " + next);
       }
@@ -264,90 +259,85 @@
     }
     nextLogMaker = ThreadPools.getServerThreadPools().getPoolBuilder("WALog creator")
         .numCoreThreads(1).enableThreadPoolMetrics().build();
-    nextLogMaker.execute(new Runnable() {
-      @Override
-      public void run() {
-        final ServerResources conf = tserver.getServerConfig();
-        final VolumeManager fs = conf.getVolumeManager();
-        while (!nextLogMaker.isShutdown()) {
-          log.debug("Creating next WAL");
-          DfsLogger alog = null;
+    nextLogMaker.execute(() -> {
+      final VolumeManager fs = tserver.getVolumeManager();
+      while (!nextLogMaker.isShutdown()) {
+        log.debug("Creating next WAL");
+        DfsLogger alog = null;
 
-          try {
-            alog = new DfsLogger(tserver.getContext(), conf, syncCounter, flushCounter);
-            alog.open(tserver.getClientAddressString());
-          } catch (Exception t) {
-            log.error("Failed to open WAL", t);
-            // the log is not advertised in ZK yet, so we can just delete it if it exists
-            if (alog != null) {
-              try {
-                alog.close();
-              } catch (Exception e) {
-                log.error("Failed to close WAL after it failed to open", e);
-              }
-
-              try {
-                Path path = alog.getPath();
-                if (fs.exists(path)) {
-                  fs.delete(path);
-                }
-              } catch (Exception e) {
-                log.warn("Failed to delete a WAL that failed to open", e);
-              }
-            }
-
+        try {
+          alog = DfsLogger.createNew(tserver.getContext(), syncCounter, flushCounter,
+              tserver.getClientAddressString());
+        } catch (Exception t) {
+          log.error("Failed to open WAL", t);
+          // the log is not advertised in ZK yet, so we can just delete it if it exists
+          if (alog != null) {
             try {
-              nextLog.offer(t, 12, TimeUnit.HOURS);
-            } catch (InterruptedException ex) {
-              // ignore
-            }
-
-            continue;
-          }
-
-          String fileName = alog.getFileName();
-          log.debug("Created next WAL {}", fileName);
-
-          try {
-            tserver.addNewLogMarker(alog);
-          } catch (Exception t) {
-            log.error("Failed to add new WAL marker for " + fileName, t);
-
-            try {
-              // Intentionally not deleting walog because it may have been advertised in ZK. See
-              // #949
               alog.close();
             } catch (Exception e) {
               log.error("Failed to close WAL after it failed to open", e);
             }
 
-            // it's possible the log was advertised in ZK even though we got an
-            // exception. If there's a chance the WAL marker may have been created,
-            // this will ensure it's closed. Either the close will be written and
-            // the GC will clean it up, or the tserver is about to die due to sesson
-            // expiration and the GC will also clean it up.
             try {
-              tserver.walogClosed(alog);
+              Path path = alog.getPath();
+              if (fs.exists(path)) {
+                fs.delete(path);
+              }
             } catch (Exception e) {
-              log.error("Failed to close WAL that failed to open: " + fileName, e);
+              log.warn("Failed to delete a WAL that failed to open", e);
             }
-
-            try {
-              nextLog.offer(t, 12, TimeUnit.HOURS);
-            } catch (InterruptedException ex) {
-              // ignore
-            }
-
-            continue;
           }
 
           try {
-            while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
-              log.info("Our WAL was not used for 12 hours: {}", fileName);
-            }
-          } catch (InterruptedException e) {
-            // ignore - server is shutting down
+            nextLog.offer(t, 12, TimeUnit.HOURS);
+          } catch (InterruptedException ex) {
+            // ignore
           }
+
+          continue;
+        }
+
+        log.debug("Created next WAL {}", alog.getLogEntry());
+
+        try {
+          tserver.addNewLogMarker(alog);
+        } catch (Exception t) {
+          log.error("Failed to add new WAL marker for " + alog.getLogEntry(), t);
+
+          try {
+            // Intentionally not deleting walog because it may have been advertised in ZK. See
+            // #949
+            alog.close();
+          } catch (Exception e) {
+            log.error("Failed to close WAL after it failed to open", e);
+          }
+
+          // it's possible the log was advertised in ZK even though we got an
+          // exception. If there's a chance the WAL marker may have been created,
+          // this will ensure it's closed. Either the close will be written and
+          // the GC will clean it up, or the tserver is about to die due to sesson
+          // expiration and the GC will also clean it up.
+          try {
+            tserver.walogClosed(alog);
+          } catch (Exception e) {
+            log.error("Failed to close WAL that failed to open: " + alog.getLogEntry(), e);
+          }
+
+          try {
+            nextLog.offer(t, 12, TimeUnit.HOURS);
+          } catch (InterruptedException ex) {
+            // ignore
+          }
+
+          continue;
+        }
+
+        try {
+          while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
+            log.info("Our WAL was not used for 12 hours: {}", alog.getLogEntry());
+          }
+        } catch (InterruptedException e) {
+          // ignore - server is shutting down
         }
       }
     });
@@ -364,7 +354,7 @@
         } catch (DfsLogger.LogClosedException ex) {
           // ignore
         } catch (Exception ex) {
-          log.error("Unable to cleanly close log " + currentLog.getFileName() + ": " + ex, ex);
+          log.error("Unable to cleanly close log " + currentLog.getLogEntry() + ": " + ex, ex);
         } finally {
           this.tserver.walogClosed(currentLog);
           currentLog = null;
@@ -406,23 +396,6 @@
               } finally {
                 commitSession.finishUpdatingLogsUsed();
               }
-
-              // Need to release
-              KeyExtent extent = commitSession.getExtent();
-              @SuppressWarnings("deprecation")
-              boolean replicationEnabled =
-                  org.apache.accumulo.core.replication.ReplicationConfigurationUtil
-                      .isEnabled(extent, tserver.getTableConfiguration(extent));
-              if (replicationEnabled) {
-                @SuppressWarnings("deprecation")
-                Status status = org.apache.accumulo.server.replication.StatusUtil
-                    .openWithUnknownLength(System.currentTimeMillis());
-                log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for "
-                    + copy.getFileName());
-                // Got some new WALs, note this in the metadata table
-                ReplicationTableUtil.updateFiles(tserver.getContext(), commitSession.getExtent(),
-                    copy.getFileName(), status);
-              }
             }
           }
         }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
index 8f2fb08..34cc101 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
@@ -191,11 +191,6 @@
     byte[] magic3 = DfsLogger.LOG_FILE_HEADER_V3.getBytes(UTF_8);
     byte[] noCryptoBytes = new NoFileEncrypter().getDecryptionParameters();
 
-    if (magic4.length != magic3.length) {
-      throw new AssertionError("Always expect log file headers to be same length : " + magic4.length
-          + " != " + magic3.length);
-    }
-
     byte[] magicBuffer = new byte[magic4.length];
     try {
       input.readFully(magicBuffer);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
index e3c6bb5..59c66e1 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
@@ -72,10 +72,6 @@
   }
 
   public CompactionExecutorsMetrics() {
-    startUpdateThread();
-  }
-
-  protected void startUpdateThread() {
     ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools()
         .createScheduledExecutorService(1, "compactionExecutorsMetricsPoller");
     Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
index bf22d1d..8cc1353 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java
@@ -41,6 +41,8 @@
   private Counter continueScanCalls;
   private Counter closeScanCalls;
   private Counter busyTimeoutCount;
+  private Counter pausedForMemory;
+  private Counter earlyReturnForMemory;
 
   private final LongAdder lookupCount = new LongAdder();
   private final LongAdder queryResultCount = new LongAdder();
@@ -71,10 +73,6 @@
     return this.queryResultBytes.sum();
   }
 
-  public void incrementScannedCount(long amount) {
-    this.scannedCount.add(amount);
-  }
-
   public LongAdder getScannedCounter() {
     return this.scannedCount;
   }
@@ -119,6 +117,14 @@
     busyTimeoutCount.increment(value);
   }
 
+  public void incrementScanPausedForLowMemory() {
+    pausedForMemory.increment();
+  }
+
+  public void incrementEarlyReturnForLowMemory() {
+    earlyReturnForMemory.increment();
+  }
+
   @Override
   public void registerMetrics(MeterRegistry registry) {
     Gauge.builder(METRICS_SCAN_OPEN_FILES, openFiles::get)
@@ -148,6 +154,11 @@
         .description("Query rate (bytes/sec)").register(registry);
     Gauge.builder(METRICS_SCAN_SCANNED_ENTRIES, this, TabletServerScanMetrics::getScannedCount)
         .description("Scanned rate").register(registry);
+    pausedForMemory = Counter.builder(METRICS_SCAN_PAUSED_FOR_MEM)
+        .description("scan paused due to server being low on memory").register(registry);
+    earlyReturnForMemory = Counter.builder(METRICS_SCAN_RETURN_FOR_MEM)
+        .description("scan returned results early due to server being low on memory")
+        .register(registry);
   }
 
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
deleted file mode 100644
index aa78dfc..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ /dev/null
@@ -1,754 +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.tserver.replication;
-
-import static java.util.Objects.requireNonNull;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-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.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.crypto.CryptoEnvironmentImpl;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.file.rfile.RFile;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.Exec;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.singletons.SingletonReservation;
-import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.threads.Threads;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.ReplicaSystem;
-import org.apache.accumulo.server.replication.ReplicaSystemHelper;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.tserver.log.DfsLogger;
-import org.apache.accumulo.tserver.log.DfsLogger.LogHeaderIncompleteException;
-import org.apache.accumulo.tserver.logger.LogFileKey;
-import org.apache.accumulo.tserver.logger.LogFileValue;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
-
-@Deprecated
-public class AccumuloReplicaSystem implements ReplicaSystem {
-
-  private static final Logger log = LoggerFactory.getLogger(AccumuloReplicaSystem.class);
-  private static final String RFILE_SUFFIX = "." + RFile.EXTENSION;
-
-  private String instanceName, zookeepers;
-  private AccumuloConfiguration conf;
-  private ServerContext context;
-
-  protected void setConf(AccumuloConfiguration conf) {
-    this.conf = conf;
-  }
-
-  /**
-   * Generate the configuration string for this ReplicaSystem
-   */
-  public static String buildConfiguration(String instanceName, String zookeepers) {
-    return instanceName + "," + zookeepers;
-  }
-
-  @Override
-  public void configure(ServerContext context, String configuration) {
-    requireNonNull(configuration);
-
-    // instance_name,zookeepers
-    int index = configuration.indexOf(',');
-    if (index == -1) {
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
-      throw new IllegalArgumentException("Expected comma in configuration string");
-    }
-
-    instanceName = configuration.substring(0, index);
-    zookeepers = configuration.substring(index + 1);
-    conf = context.getConfiguration();
-    this.context = context;
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by admin")
-  @Override
-  public Status replicate(final Path p, final Status status, final ReplicationTarget target,
-      final ReplicaSystemHelper helper) {
-    final AccumuloConfiguration localConf = conf;
-
-    log.debug("Replication RPC timeout is {}", localConf.get(Property.REPLICATION_RPC_TIMEOUT));
-
-    final String principal = getPrincipal(localConf, target);
-    final File keytab;
-    final String password;
-    if (localConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
-      String keytabPath = getKeytab(localConf, target);
-      keytab = new File(keytabPath);
-      if (!keytab.exists() || !keytab.isFile()) {
-        log.error("{} is not a regular file. Cannot login to replicate", keytabPath);
-        return status;
-      }
-      password = null;
-    } else {
-      keytab = null;
-      password = getPassword(localConf, target);
-    }
-
-    if (keytab != null) {
-      try {
-        final UserGroupInformation accumuloUgi = UserGroupInformation.getCurrentUser();
-        // Get a UGI with the principal + keytab
-        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal,
-            keytab.getAbsolutePath());
-
-        // Run inside a doAs to avoid nuking the Tserver's user
-        return ugi.doAs((PrivilegedAction<Status>) () -> {
-          KerberosToken token;
-          try {
-            // Do *not* replace the current user
-            token = new KerberosToken(principal, keytab);
-          } catch (IOException e) {
-            log.error("Failed to create KerberosToken", e);
-            return status;
-          }
-          ClientContext peerContext = getContextForPeer(localConf, target, principal, token);
-          return _replicate(p, status, target, helper, localConf, peerContext, accumuloUgi);
-        });
-      } catch (IOException e) {
-        // Can't log in, can't replicate
-        log.error("Failed to perform local login", e);
-        return status;
-      }
-    } else {
-      // Simple case: make a password token, context and then replicate
-      PasswordToken token = new PasswordToken(password);
-      ClientContext peerContext = getContextForPeer(localConf, target, principal, token);
-      return _replicate(p, status, target, helper, localConf, peerContext, null);
-    }
-  }
-
-  /**
-   * Perform replication, making a few attempts when an exception is returned.
-   *
-   * @param p Path of WAL to replicate
-   * @param status Current status for the WAL
-   * @param target Where we're replicating to
-   * @param helper A helper for replication
-   * @param localConf The local instance's configuration
-   * @param peerContext The ClientContext to connect to the peer
-   * @return The new (or unchanged) Status for the WAL
-   */
-  private Status _replicate(final Path p, final Status status, final ReplicationTarget target,
-      final ReplicaSystemHelper helper, final AccumuloConfiguration localConf,
-      final ClientContext peerContext, final UserGroupInformation accumuloUgi) {
-
-    Span span = TraceUtil.startSpan(this.getClass(), "_replicate");
-    try (Scope replicaScope = span.makeCurrent()) {
-
-      // Remote identifier is an integer (table id) in this case.
-      final String remoteTableId = target.getRemoteIdentifier();
-
-      // Attempt the replication of this status a number of times before giving up and
-      // trying to replicate it again later some other time.
-      int numAttempts = localConf.getCount(Property.REPLICATION_WORK_ATTEMPTS);
-      for (int i = 0; i < numAttempts; i++) {
-        log.debug("Attempt {}", i);
-        String peerTserverStr;
-        log.debug("Fetching peer tserver address");
-        Span span2 = TraceUtil.startSpan(this.getClass(), "_replicate::Fetch peer tserver");
-        try (Scope scope = span2.makeCurrent()) {
-          // Ask the manager on the remote what TServer we should talk with to replicate the data
-          peerTserverStr = ThriftClientTypes.REPLICATION_COORDINATOR.execute(peerContext,
-              client -> client.getServicerAddress(remoteTableId, peerContext.rpcCreds()));
-        } catch (AccumuloException | AccumuloSecurityException e) {
-          // No progress is made
-          log.error(
-              "Could not connect to manager at {}, cannot proceed with replication. Will retry",
-              target, e);
-          TraceUtil.setException(span2, e, false);
-          continue;
-        } catch (RuntimeException e) {
-          TraceUtil.setException(span2, e, true);
-          throw e;
-        } finally {
-          span2.end();
-        }
-
-        if (peerTserverStr == null) {
-          // Something went wrong, and we didn't get a valid tserver from the remote for some reason
-          log.warn("Did not receive tserver from manager at {}, cannot proceed"
-              + " with replication. Will retry.", target);
-          continue;
-        }
-
-        final HostAndPort peerTserver = HostAndPort.fromString(peerTserverStr);
-
-        final long timeout = localConf.getTimeInMillis(Property.REPLICATION_RPC_TIMEOUT);
-
-        // We have a tserver on the remote -- send the data its way.
-        Status finalStatus;
-        final long sizeLimit = conf.getAsBytes(Property.REPLICATION_MAX_UNIT_SIZE);
-        try {
-          if (p.getName().endsWith(RFILE_SUFFIX)) {
-            Span span3 = TraceUtil.startSpan(this.getClass(), "_replicate::RFile replication");
-            try (Scope scope = span3.makeCurrent()) {
-              finalStatus = replicateRFiles(peerContext, peerTserver, target, p, status, timeout);
-            } catch (RuntimeException e) {
-              TraceUtil.setException(span3, e, true);
-              throw e;
-            } finally {
-              span3.end();
-            }
-          } else {
-            Span span4 = TraceUtil.startSpan(this.getClass(), "_replicate::WAL replication");
-            try (Scope scope = span4.makeCurrent()) {
-              finalStatus = replicateLogs(peerContext, peerTserver, target, p, status, sizeLimit,
-                  remoteTableId, peerContext.rpcCreds(), helper, accumuloUgi, timeout);
-            } catch (RuntimeException e) {
-              TraceUtil.setException(span4, e, true);
-              throw e;
-            } finally {
-              span4.end();
-            }
-          }
-
-          log.debug("New status for {} after replicating to {} is {}", p,
-              peerContext.getInstanceName(), ProtobufUtil.toString(finalStatus));
-
-          return finalStatus;
-        } catch (TTransportException | AccumuloException | AccumuloSecurityException e) {
-          log.warn("Could not connect to remote server {}, will retry", peerTserverStr, e);
-          TraceUtil.setException(span, e, false);
-          sleepUninterruptibly(1, TimeUnit.SECONDS);
-        }
-      }
-
-      log.info("No progress was made after {} attempts to replicate {},"
-          + " returning so file can be re-queued", numAttempts, p);
-
-      // We made no status, punt on it for now, and let it re-queue itself for work
-      return status;
-    } catch (RuntimeException e) {
-      TraceUtil.setException(span, e, true);
-      throw e;
-    } finally {
-      span.end();
-    }
-  }
-
-  protected Status replicateRFiles(ClientContext peerContext, final HostAndPort peerTserver,
-      final ReplicationTarget target, final Path p, final Status status, long timeout)
-      throws TTransportException, AccumuloException, AccumuloSecurityException {
-
-    Status lastStatus = status, currentStatus = status;
-    while (true) {
-      // Read and send a batch of mutations
-      ReplicationStats replResult =
-          executeServicerWithReturn(peerContext, peerTserver, new RFileClientExecReturn(), timeout);
-
-      // Catch the overflow
-      long newBegin = currentStatus.getBegin() + replResult.entriesConsumed;
-      if (newBegin < 0) {
-        newBegin = Long.MAX_VALUE;
-      }
-
-      currentStatus = Status.newBuilder(currentStatus).setBegin(newBegin).build();
-
-      log.debug("Sent batch for replication of {} to {}, with new Status {}", p, target,
-          ProtobufUtil.toString(currentStatus));
-
-      // If we got a different status
-      if (currentStatus.equals(lastStatus)) {
-        log.debug("Did not replicate any new data for {} to {}, (state was {})", p, target,
-            ProtobufUtil.toString(lastStatus));
-
-        // otherwise, we didn't actually replicate (likely because there was error sending the
-        // data)
-        // we can just not record any updates, and it will be picked up again by the work assigner
-        return status;
-      } else {
-        // If we don't have any more work, just quit
-        if (StatusUtil.isWorkRequired(currentStatus)) {
-          // Otherwise, let it loop and replicate some more data
-          lastStatus = currentStatus;
-        } else {
-          return currentStatus;
-        }
-      }
-    }
-  }
-
-  protected Status replicateLogs(ClientContext peerContext, final HostAndPort peerTserver,
-      final ReplicationTarget target, final Path p, final Status status, final long sizeLimit,
-      final String remoteTableId, final TCredentials tcreds, final ReplicaSystemHelper helper,
-      final UserGroupInformation accumuloUgi, long timeout)
-      throws TTransportException, AccumuloException, AccumuloSecurityException {
-
-    log.debug("Replication WAL to peer tserver");
-    final Set<Integer> tids;
-    try (final FSDataInputStream fsinput = context.getVolumeManager().open(p);
-        final DataInputStream input = getWalStream(p, fsinput)) {
-      log.debug("Skipping unwanted data in WAL");
-      Span span = TraceUtil.startSpan(this.getClass(), "replicateLogs::Consume WAL prefix");
-      try (Scope scope = span.makeCurrent()) {
-        span.setAttribute("file", p.toString());
-        // We want to read all records in the WAL up to the "begin" offset contained in the Status
-        // message,
-        // building a Set of tids from DEFINE_TABLET events which correspond to table ids for future
-        // mutations
-        tids = consumeWalPrefix(target, input, status);
-      } catch (IOException e) {
-        log.warn("Unexpected error consuming file.");
-        TraceUtil.setException(span, e, false);
-        return status;
-      } catch (RuntimeException e) {
-        TraceUtil.setException(span, e, true);
-        throw e;
-      } finally {
-        span.end();
-      }
-
-      log.debug("Sending batches of data to peer tserver");
-
-      Status lastStatus = status, currentStatus = status;
-      final AtomicReference<Exception> exceptionRef = new AtomicReference<>();
-      while (true) {
-        ReplicationStats replResult;
-        Span span2 = TraceUtil.startSpan(this.getClass(), "replicateLogs::Replicate WAL batch");
-        try (Scope scope = span2.makeCurrent()) {
-          span2.setAttribute("Batch size (bytes)", Long.toString(sizeLimit));
-          span2.setAttribute("File", p.toString());
-          span2.setAttribute("Peer instance name", peerContext.getInstanceName());
-          span2.setAttribute("Peer tserver", peerTserver.toString());
-          span2.setAttribute("Remote table ID", remoteTableId);
-
-          // Read and send a batch of mutations
-          replResult = executeServicerWithReturn(peerContext, peerTserver,
-              new WalClientExecReturn(this, target, input, p, currentStatus, sizeLimit,
-                  remoteTableId, tcreds, tids),
-              timeout);
-        } catch (RuntimeException e) {
-          log.error("Caught exception replicating data to {} at {}", peerContext.getInstanceName(),
-              peerTserver, e);
-          TraceUtil.setException(span2, e, true);
-          throw e;
-        } finally {
-          span2.end();
-        }
-
-        // Catch the overflow
-        long newBegin = currentStatus.getBegin() + replResult.entriesConsumed;
-        if (newBegin < 0) {
-          newBegin = Long.MAX_VALUE;
-        }
-
-        currentStatus = Status.newBuilder(currentStatus).setBegin(newBegin).build();
-
-        log.debug("Sent batch for replication of {} to {}, with new Status {}", p, target,
-            ProtobufUtil.toString(currentStatus));
-
-        // If we got a different status
-        if (currentStatus.equals(lastStatus)) {
-          log.debug("Did not replicate any new data for {} to {}, (state was {})", p, target,
-              ProtobufUtil.toString(lastStatus));
-
-          // otherwise, we didn't actually replicate (likely because there was error sending the
-          // data)
-          // we can just not record any updates, and it will be picked up again by the work assigner
-          return status;
-        } else {
-          Span span3 =
-              TraceUtil.startSpan(this.getClass(), "replicateLogs::Update replication table");
-          try (Scope scope = span3.makeCurrent()) {
-            if (accumuloUgi != null) {
-              final Status copy = currentStatus;
-              accumuloUgi.doAs((PrivilegedAction<Void>) () -> {
-                try {
-                  helper.recordNewStatus(p, copy, target);
-                } catch (TableNotFoundException | AccumuloException | RuntimeException e) {
-                  exceptionRef.set(e);
-                }
-                return null;
-              });
-              Exception e = exceptionRef.get();
-              if (e != null) {
-                if (e instanceof TableNotFoundException) {
-                  throw (TableNotFoundException) e;
-                } else if (e instanceof AccumuloSecurityException) {
-                  throw (AccumuloSecurityException) e;
-                } else if (e instanceof AccumuloException) {
-                  throw (AccumuloException) e;
-                } else {
-                  throw new RuntimeException("Received unexpected exception", e);
-                }
-              }
-            } else {
-              helper.recordNewStatus(p, currentStatus, target);
-            }
-          } catch (TableNotFoundException e) {
-            log.error(
-                "Tried to update status in replication table for {} as"
-                    + " {}, but the table did not exist",
-                p, ProtobufUtil.toString(currentStatus), e);
-            TraceUtil.setException(span3, e, true);
-            throw new RuntimeException("Replication table did not exist, will retry", e);
-          } catch (RuntimeException e) {
-            TraceUtil.setException(span3, e, true);
-            throw e;
-          } finally {
-            span3.end();
-          }
-
-          log.debug("Recorded updated status for {}: {}", p, ProtobufUtil.toString(currentStatus));
-
-          // If we don't have any more work, just quit
-          if (StatusUtil.isWorkRequired(currentStatus)) {
-            // Otherwise, let it loop and replicate some more data
-            lastStatus = currentStatus;
-          } else {
-            return currentStatus;
-          }
-        }
-      }
-    } catch (LogHeaderIncompleteException e) {
-      log.warn("Could not read header from {}, assuming that there is no data"
-          + " present in the WAL, therefore replication is complete", p);
-      Status newStatus;
-      // Bump up the begin to the (infinite) end, trying to be accurate
-      if (status.getInfiniteEnd()) {
-        newStatus = Status.newBuilder(status).setBegin(Long.MAX_VALUE).build();
-      } else {
-        newStatus = Status.newBuilder(status).setBegin(status.getEnd()).build();
-      }
-      Span span4 = TraceUtil.startSpan(this.getClass(), "replicateLogs::Update replication table");
-      try (Scope scope = span4.makeCurrent()) {
-        helper.recordNewStatus(p, newStatus, target);
-      } catch (TableNotFoundException tnfe) {
-        log.error(
-            "Tried to update status in replication table for {} as {}, but the table did not exist",
-            p, ProtobufUtil.toString(newStatus), e);
-        TraceUtil.setException(span4, e, true);
-        throw new RuntimeException("Replication table did not exist, will retry", e);
-      } catch (RuntimeException ex) {
-        TraceUtil.setException(span4, e, true);
-        throw ex;
-      } finally {
-        span4.end();
-      }
-      return newStatus;
-    } catch (IOException e) {
-      log.error("Could not create stream for WAL", e);
-      // No data sent (bytes nor records) and no progress made
-      return status;
-    }
-  }
-
-  @Deprecated
-  protected static class RFileClientExecReturn
-      implements Exec<ReplicationStats,ReplicationServicer.Client> {
-
-    @Override
-    public ReplicationStats execute(Client client) {
-      // No data sent (bytes nor records) and no progress made
-      return new ReplicationStats(0L, 0L, 0L);
-    }
-  }
-
-  protected String getPassword(AccumuloConfiguration localConf, ReplicationTarget target) {
-    requireNonNull(localConf);
-    requireNonNull(target);
-
-    Map<String,String> peerPasswords =
-        localConf.getAllPropertiesWithPrefix(Property.REPLICATION_PEER_PASSWORD);
-    String password =
-        peerPasswords.get(Property.REPLICATION_PEER_PASSWORD.getKey() + target.getPeerName());
-    if (password == null) {
-      throw new IllegalArgumentException("Cannot get password for " + target.getPeerName());
-    }
-    return password;
-  }
-
-  protected String getKeytab(AccumuloConfiguration localConf, ReplicationTarget target) {
-    requireNonNull(localConf);
-    requireNonNull(target);
-
-    Map<String,String> peerKeytabs =
-        localConf.getAllPropertiesWithPrefix(Property.REPLICATION_PEER_KEYTAB);
-    String keytab =
-        peerKeytabs.get(Property.REPLICATION_PEER_KEYTAB.getKey() + target.getPeerName());
-    if (keytab == null) {
-      throw new IllegalArgumentException("Cannot get keytab for " + target.getPeerName());
-    }
-    return keytab;
-  }
-
-  protected String getPrincipal(AccumuloConfiguration localConf, ReplicationTarget target) {
-    requireNonNull(localConf);
-    requireNonNull(target);
-
-    String peerName = target.getPeerName();
-    String userKey = Property.REPLICATION_PEER_USER.getKey() + peerName;
-    Map<String,String> peerUsers =
-        localConf.getAllPropertiesWithPrefix(Property.REPLICATION_PEER_USER);
-
-    String user = peerUsers.get(userKey);
-    if (user == null) {
-      throw new IllegalArgumentException("Cannot get user for " + target.getPeerName());
-    }
-    return user;
-  }
-
-  protected ClientContext getContextForPeer(AccumuloConfiguration localConf,
-      ReplicationTarget target, String principal, AuthenticationToken token) {
-    requireNonNull(localConf);
-    requireNonNull(target);
-    requireNonNull(principal);
-    requireNonNull(token);
-
-    Properties properties = new Properties();
-    properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), instanceName);
-    properties.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), zookeepers);
-    properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
-    ClientProperty.setAuthenticationToken(properties, token);
-
-    return new ClientContext(SingletonReservation.noop(), ClientInfo.from(properties, token),
-        localConf, Threads.UEH);
-  }
-
-  protected Set<Integer> consumeWalPrefix(ReplicationTarget target, DataInputStream wal,
-      Status status) throws IOException {
-    LogFileKey key = new LogFileKey();
-    LogFileValue value = new LogFileValue();
-
-    Set<Integer> desiredTids = new HashSet<>();
-
-    // Read through the stuff we've already processed in a previous replication attempt
-    // We also need to track the tids that occurred earlier in the file as mutations
-    // later on might use that tid
-    for (long i = 0; i < status.getBegin(); i++) {
-      key.readFields(wal);
-      value.readFields(wal);
-
-      switch (key.event) {
-        case DEFINE_TABLET:
-          if (target.getSourceTableId().equals(key.tablet.tableId())) {
-            desiredTids.add(key.tabletId);
-          }
-          break;
-        default:
-          break;
-      }
-    }
-
-    return desiredTids;
-  }
-
-  public DataInputStream getWalStream(Path p, FSDataInputStream input)
-      throws LogHeaderIncompleteException, IOException {
-    Span span = TraceUtil.startSpan(this.getClass(), "getWalStream::Read WAL header");
-    try (Scope scope = span.makeCurrent()) {
-      span.setAttribute("file", p.toString());
-      CryptoEnvironment env = new CryptoEnvironmentImpl(CryptoEnvironment.Scope.WAL);
-      return DfsLogger.getDecryptingStream(input,
-          context.getCryptoFactory().getService(env, conf.getAllCryptoProperties()));
-    } catch (RuntimeException e) {
-      TraceUtil.setException(span, e, true);
-      throw e;
-    } finally {
-      span.end();
-    }
-  }
-
-  protected WalReplication getWalEdits(ReplicationTarget target, DataInputStream wal, Path p,
-      Status status, long sizeLimit, Set<Integer> desiredTids) {
-    WalEdits edits = new WalEdits();
-    edits.edits = new ArrayList<>();
-    long size = 0L;
-    long entriesConsumed = 0L;
-    long numUpdates = 0L;
-    LogFileKey key = new LogFileKey();
-    LogFileValue value = new LogFileValue();
-
-    while (size < sizeLimit) {
-      try {
-        key.readFields(wal);
-        value.readFields(wal);
-      } catch (EOFException e) {
-        log.debug("Caught EOFException reading {}", p, e);
-        if (status.getInfiniteEnd() && status.getClosed()) {
-          log.debug("{} is closed and has unknown length, assuming entire file has been consumed",
-              p);
-          entriesConsumed = Long.MAX_VALUE;
-        }
-        break;
-      } catch (IOException e) {
-        log.debug("Unexpected IOException reading {}", p, e);
-        throw new UncheckedIOException(e);
-      }
-
-      entriesConsumed++;
-
-      switch (key.event) {
-        case DEFINE_TABLET:
-          // For new DEFINE_TABLETs, we also need to record the new tids we see
-          if (target.getSourceTableId().equals(key.tablet.tableId())) {
-            desiredTids.add(key.tabletId);
-          }
-          break;
-        case MUTATION:
-        case MANY_MUTATIONS:
-          // Only write out mutations for tids that are for the desired tablet
-          if (desiredTids.contains(key.tabletId)) {
-
-            byte[] data;
-            try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                DataOutputStream out = new DataOutputStream(baos)) {
-              key.write(out);
-
-              // Only write out the mutations that don't have the given ReplicationTarget
-              // as a replicate source (this prevents infinite replication loops: a->b, b->a,
-              // repeat)
-              numUpdates += writeValueAvoidingReplicationCycles(out, value, target);
-
-              out.flush();
-              data = baos.toByteArray();
-            } catch (IOException e) {
-              log.debug("Unexpected IOException writing to a byte array output stream", e);
-              throw new UncheckedIOException(e);
-            }
-            size += data.length;
-            edits.addToEdits(ByteBuffer.wrap(data));
-          }
-          break;
-        default:
-          log.trace("Ignoring WAL entry which doesn't contain mutations,"
-              + " should not have received such entries");
-          break;
-      }
-    }
-
-    return new WalReplication(edits, size, entriesConsumed, numUpdates);
-  }
-
-  /**
-   * Wrapper around {@link LogFileValue#write(java.io.DataOutput)} which does not serialize
-   * {@link Mutation}s that do not need to be replicate to the given {@link ReplicationTarget}
-   */
-  protected long writeValueAvoidingReplicationCycles(DataOutputStream out, LogFileValue value,
-      ReplicationTarget target) throws IOException {
-    // TODO This works like LogFileValue, and needs to be parsable by it, which makes this
-    // serialization brittle.
-    // see matching TODO in BatchWriterReplicationReplayer
-
-    int mutationsToSend = 0;
-    for (Mutation m : value.mutations) {
-      if (!m.getReplicationSources().contains(target.getPeerName())) {
-        mutationsToSend++;
-      }
-    }
-
-    int mutationsRemoved = value.mutations.size() - mutationsToSend;
-    if (mutationsRemoved > 0) {
-      log.debug("Removing {} mutations from WAL entry as they have already been replicated to {}",
-          mutationsRemoved, target.getPeerName());
-    }
-
-    // Add our name, and send it
-    final String name = conf.get(Property.REPLICATION_NAME);
-    if (name.isBlank()) {
-      throw new IllegalArgumentException("Local system has no replication name configured");
-    }
-
-    out.writeInt(mutationsToSend);
-    for (Mutation m : value.mutations) {
-      // If we haven't yet replicated to this peer
-      if (!m.getReplicationSources().contains(target.getPeerName())) {
-        m.addReplicationSource(name);
-
-        m.write(out);
-      }
-    }
-
-    return mutationsToSend;
-  }
-
-  private static <T> T executeServicerWithReturn(ClientContext context, HostAndPort tserver,
-      Exec<T,ReplicationServicer.Client> exec, long timeout)
-      throws AccumuloException, AccumuloSecurityException {
-    ReplicationServicer.Client client = null;
-    try {
-      client =
-          ThriftUtil.getClient(ThriftClientTypes.REPLICATION_SERVICER, tserver, context, timeout);
-      return exec.execute(client);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (TException e) {
-      throw new AccumuloException(e);
-    } finally {
-      if (client != null) {
-        ThriftUtil.close(client, context);
-      }
-    }
-  }
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
deleted file mode 100644
index 86db3b2..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
+++ /dev/null
@@ -1,174 +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.tserver.replication;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.replication.AccumuloReplicationReplayer;
-import org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode;
-import org.apache.accumulo.core.replication.thrift.RemoteReplicationException;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.server.data.ServerMutation;
-import org.apache.accumulo.tserver.logger.LogFileKey;
-import org.apache.accumulo.tserver.logger.LogFileValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Use a BatchWriter to replay WAL entries to an Accumulo table. This assumes that all WAL entries
- * are for this table. Pruning out undesired entries is expected to be done by the sender.
- */
-@Deprecated
-public class BatchWriterReplicationReplayer implements AccumuloReplicationReplayer {
-  private static final Logger log = LoggerFactory.getLogger(BatchWriterReplicationReplayer.class);
-
-  @Override
-  public long replicateLog(ClientContext context, String tableName, WalEdits data)
-      throws RemoteReplicationException {
-    final LogFileKey key = new LogFileKey();
-    final LogFileValue value = new LogFileValue();
-    final long memoryInBytes =
-        context.getConfiguration().getAsBytes(Property.TSERV_REPLICATION_BW_REPLAYER_MEMORY);
-
-    BatchWriter bw = null;
-    long mutationsApplied = 0L;
-    try {
-      for (ByteBuffer edit : data.getEdits()) {
-        DataInputStream dis = new DataInputStream(ByteBufferUtil.toByteArrayInputStream(edit));
-        try {
-          key.readFields(dis);
-          // TODO this is brittle because AccumuloReplicaSystem isn't actually calling
-          // LogFileValue.write, but we're expecting
-          // what we receive to be readable by the LogFileValue.
-          value.readFields(dis);
-        } catch (IOException e) {
-          log.error("Could not deserialize edit from stream", e);
-          throw new RemoteReplicationException(RemoteReplicationErrorCode.COULD_NOT_DESERIALIZE,
-              "Could not deserialize edit from stream");
-        }
-
-        // Create the batchScanner if we don't already have one.
-        if (bw == null) {
-          BatchWriterConfig bwConfig = new BatchWriterConfig();
-          bwConfig.setMaxMemory(memoryInBytes);
-          try {
-            bw = context.createBatchWriter(tableName, bwConfig);
-          } catch (TableNotFoundException e) {
-            throw new RemoteReplicationException(RemoteReplicationErrorCode.TABLE_DOES_NOT_EXIST,
-                "Table " + tableName + " does not exist");
-          }
-        }
-
-        log.info("Applying {} mutations to table {} as part of batch", value.mutations.size(),
-            tableName);
-
-        // If we got a ServerMutation, we have to make sure that we preserve the systemTimestamp
-        // otherwise
-        // the local system will assign a new timestamp.
-        List<Mutation> mutationsCopy = new ArrayList<>(value.mutations.size());
-        long mutationsCopied = 0L;
-        for (Mutation orig : value.mutations) {
-          if (orig instanceof ServerMutation) {
-            mutationsCopied++;
-
-            ServerMutation origServer = (ServerMutation) orig;
-            Mutation copy = new Mutation(orig.getRow());
-            for (ColumnUpdate update : orig.getUpdates()) {
-              long timestamp;
-
-              // If the update doesn't have a timestamp, pull it from the ServerMutation
-              if (update.hasTimestamp()) {
-                timestamp = update.getTimestamp();
-              } else {
-                timestamp = origServer.getSystemTimestamp();
-              }
-
-              // TODO ACCUMULO-2937 cache the CVs
-              if (update.isDeleted()) {
-                copy.putDelete(update.getColumnFamily(), update.getColumnQualifier(),
-                    new ColumnVisibility(update.getColumnVisibility()), timestamp);
-              } else {
-                copy.put(update.getColumnFamily(), update.getColumnQualifier(),
-                    new ColumnVisibility(update.getColumnVisibility()), timestamp,
-                    update.getValue());
-              }
-            }
-
-            // We also need to preserve the replicationSource information to prevent cycles
-            Set<String> replicationSources = orig.getReplicationSources();
-            if (replicationSources != null && !replicationSources.isEmpty()) {
-              for (String replicationSource : replicationSources) {
-                copy.addReplicationSource(replicationSource);
-              }
-            }
-
-            mutationsCopy.add(copy);
-          } else {
-            mutationsCopy.add(orig);
-          }
-        }
-
-        log.debug("Copied {} mutations to ensure server-assigned timestamps are propagated",
-            mutationsCopied);
-
-        try {
-          bw.addMutations(mutationsCopy);
-        } catch (MutationsRejectedException e) {
-          log.error("Could not apply mutations to {}", tableName);
-          throw new RemoteReplicationException(RemoteReplicationErrorCode.COULD_NOT_APPLY,
-              "Could not apply mutations to " + tableName);
-        }
-
-        log.debug("{} mutations added to the BatchScanner", mutationsCopy.size());
-
-        mutationsApplied += mutationsCopy.size();
-      }
-    } finally {
-      if (bw != null) {
-        try {
-          bw.close();
-        } catch (MutationsRejectedException e) {
-          log.error("Could not apply mutations to {}", tableName);
-          throw new RemoteReplicationException(RemoteReplicationErrorCode.COULD_NOT_APPLY,
-              "Could not apply mutations to " + tableName);
-        }
-      }
-    }
-
-    log.info("Applied {} mutations in total to {}", mutationsApplied, tableName);
-
-    return mutationsApplied;
-  }
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
deleted file mode 100644
index 1c3327c..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
+++ /dev/null
@@ -1,182 +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.tserver.replication;
-
-import static com.google.common.collect.MoreCollectors.onlyElement;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.NoSuchElementException;
-
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.ReplicaSystem;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.server.replication.ReplicaSystemHelper;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue.Processor;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-/**
- * Transmit the given data to a peer
- */
-@Deprecated
-public class ReplicationProcessor implements Processor {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationProcessor.class);
-
-  private final ServerContext context;
-  private final ReplicaSystemHelper helper;
-  private final ReplicaSystemFactory factory;
-
-  public ReplicationProcessor(ServerContext context) {
-    this.context = context;
-    this.helper = new ReplicaSystemHelper(context);
-    this.factory = new ReplicaSystemFactory();
-  }
-
-  @Override
-  public ReplicationProcessor newProcessor() {
-    return new ReplicationProcessor(context);
-  }
-
-  @Override
-  public void process(String workID, byte[] data) {
-    ReplicationTarget target =
-        DistributedWorkQueueWorkAssignerHelper.fromQueueKey(workID).getValue();
-    String file = new String(data, UTF_8);
-
-    log.debug("Received replication work for {} to {}", file, target);
-
-    ReplicaSystem replica;
-    try {
-      replica = getReplicaSystem(target);
-    } catch (Exception e) {
-      log.error("Could not instantiate ReplicaSystem for {}, waiting before returning the work",
-          target, e);
-      try {
-        // TODO configurable
-        Thread.sleep(5000);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-      }
-
-      return;
-    }
-
-    Status status;
-    try {
-      status = getStatus(file, target);
-    } catch (ReplicationTableOfflineException e) {
-      log.error("Could not look for replication record", e);
-      throw new IllegalStateException("Could not look for replication record", e);
-    } catch (InvalidProtocolBufferException e) {
-      log.error("Could not deserialize Status from Work section for {} and {}", file, target);
-      throw new RuntimeException("Could not parse Status for work record", e);
-    } catch (NoSuchElementException e) {
-      log.error("Assigned work for {} to {} but could not find work record", file, target);
-      return;
-    }
-
-    log.debug("Current status for {} replicating to {}: {}", file, target,
-        ProtobufUtil.toString(status));
-
-    // We don't need to do anything (shouldn't have gotten this work record in the first place)
-    if (!StatusUtil.isWorkRequired(status)) {
-      log.info("Received work request for {} and {}, but it does not need replication. Ignoring...",
-          file, target);
-      return;
-    }
-
-    // Sanity check that nothing bad happened and our replication source still exists
-    Path filePath = new Path(file);
-    try {
-      if (!doesFileExist(filePath, target)) {
-        return;
-      }
-    } catch (IOException e) {
-      log.error("Could not determine if file exists {}", filePath, e);
-      throw new RuntimeException(e);
-    }
-
-    log.debug("Replicating {} to {} using {}", filePath, target, replica.getClass().getName());
-
-    Status newStatus = replica.replicate(filePath, status, target, getHelper());
-
-    log.debug("Finished replicating {}. Original status: {}, New status: {}", filePath, status,
-        newStatus);
-  }
-
-  protected ReplicaSystemHelper getHelper() {
-    return helper;
-  }
-
-  protected ReplicaSystem getReplicaSystem(ReplicationTarget target) {
-    // Find the configured replication peer so we know how to replicate to it
-    // Classname,Configuration
-    String peerType = getPeerType(target.getPeerName());
-
-    // Get the peer that we're replicating to
-    return factory.get(context, peerType);
-  }
-
-  protected String getPeerType(String peerName) {
-    // Find the configured replication peer so we know how to replicate to it
-    Map<String,String> configuredPeers =
-        context.getConfiguration().getAllPropertiesWithPrefix(Property.REPLICATION_PEERS);
-    String peerType = configuredPeers.get(Property.REPLICATION_PEERS.getKey() + peerName);
-    if (peerType == null) {
-      String msg = "Cannot process replication for unknown peer: " + peerName;
-      log.warn(msg);
-      throw new IllegalArgumentException(msg);
-    }
-
-    return peerType;
-  }
-
-  protected boolean doesFileExist(Path filePath, ReplicationTarget target) throws IOException {
-    if (!context.getVolumeManager().exists(filePath)) {
-      log.warn("Received work request for {} and {}, but the file doesn't exist", filePath, target);
-      return false;
-    }
-
-    return true;
-  }
-
-  protected Status getStatus(String file, ReplicationTarget target)
-      throws ReplicationTableOfflineException, InvalidProtocolBufferException {
-    Scanner s = ReplicationTable.getScanner(context);
-    s.setRange(Range.exact(file));
-    s.fetchColumn(WorkSection.NAME, target.toText());
-    return Status.parseFrom(s.stream().collect(onlyElement()).getValue().get());
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java
deleted file mode 100644
index ab8400d..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java
+++ /dev/null
@@ -1,125 +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.tserver.replication;
-
-import java.util.Map;
-
-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.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.replication.AccumuloReplicationReplayer;
-import org.apache.accumulo.core.replication.thrift.KeyValues;
-import org.apache.accumulo.core.replication.thrift.RemoteReplicationErrorCode;
-import org.apache.accumulo.core.replication.thrift.RemoteReplicationException;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Iface;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.tserver.TabletServer;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationServicerHandler implements Iface {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationServicerHandler.class);
-
-  private TabletServer tabletServer;
-
-  public ReplicationServicerHandler(TabletServer tabletServer) {
-    this.tabletServer = tabletServer;
-  }
-
-  @Override
-  public long replicateLog(String tableIdStr, WalEdits data, TCredentials tcreds)
-      throws TException {
-    TableId tableId = TableId.of(tableIdStr);
-    log.debug("Got replication request to tableID {} with {} edits", tableId, data.getEditsSize());
-    tabletServer.getSecurityOperation().authenticateUser(tabletServer.getContext().rpcCreds(),
-        tcreds);
-
-    String tableName;
-
-    try {
-      tableName = tabletServer.getContext().getTableName(tableId);
-    } catch (TableNotFoundException e) {
-      log.error("Could not find table with id {}", tableId);
-      throw new RemoteReplicationException(RemoteReplicationErrorCode.TABLE_DOES_NOT_EXIST,
-          "Table with id " + tableId + " does not exist");
-    }
-
-    AccumuloConfiguration conf = tabletServer.getConfiguration();
-
-    Map<String,String> replicationHandlers =
-        conf.getAllPropertiesWithPrefix(Property.TSERV_REPLICATION_REPLAYERS);
-    String propertyForHandlerTable = Property.TSERV_REPLICATION_REPLAYERS.getKey() + tableId;
-
-    String handlerClassForTable = replicationHandlers.get(propertyForHandlerTable);
-    if (handlerClassForTable == null) {
-      if (!replicationHandlers.isEmpty()) {
-        log.debug("Could not find replication replayer for {}", tableId);
-      }
-      handlerClassForTable = conf.get(Property.TSERV_REPLICATION_DEFAULT_HANDLER);
-    }
-
-    log.debug("Using {} replication replayer for table {}", handlerClassForTable, tableId);
-
-    // Get class for replayer
-    Class<? extends AccumuloReplicationReplayer> clz;
-    try {
-      Class<?> untypedClz = Class.forName(handlerClassForTable);
-      clz = untypedClz.asSubclass(AccumuloReplicationReplayer.class);
-    } catch (ClassNotFoundException e) {
-      log.error("Could not instantiate replayer class {}", handlerClassForTable, e);
-      throw new RemoteReplicationException(RemoteReplicationErrorCode.CANNOT_INSTANTIATE_REPLAYER,
-          "Could not instantiate replayer class " + handlerClassForTable);
-    }
-
-    // Create an instance
-    AccumuloReplicationReplayer replayer;
-    try {
-      replayer = clz.getDeclaredConstructor().newInstance();
-    } catch (ReflectiveOperationException e1) {
-      log.error("Could not instantiate replayer class {}", clz.getName());
-      throw new RemoteReplicationException(RemoteReplicationErrorCode.CANNOT_INSTANTIATE_REPLAYER,
-          "Could not instantiate replayer class" + clz.getName());
-    }
-
-    long entriesReplicated;
-    try {
-      entriesReplicated = replayer.replicateLog(tabletServer.getContext(), tableName, data);
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      log.error("Could not get connection", e);
-      throw new RemoteReplicationException(RemoteReplicationErrorCode.CANNOT_AUTHENTICATE,
-          "Cannot get connection as " + tabletServer.getContext().getCredentials().getPrincipal());
-    }
-
-    log.debug("Replicated {} mutations to {}", entriesReplicated, tableName);
-
-    return entriesReplicated;
-  }
-
-  @Override
-  public long replicateKeyValues(String tableId, KeyValues data, TCredentials creds) {
-    throw new UnsupportedOperationException();
-  }
-
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationStats.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationStats.java
deleted file mode 100644
index aeef0f7..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationStats.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.
- */
-package org.apache.accumulo.tserver.replication;
-
-import java.util.Objects;
-
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-
-@Deprecated
-public class ReplicationStats {
-  /**
-   * The size, in bytes, of the data sent
-   */
-  public long sizeInBytes;
-
-  /**
-   * The number of records sent
-   */
-  public long sizeInRecords;
-
-  /**
-   * The number of entries consumed from the log (to increment {@link Status}'s begin)
-   */
-  public long entriesConsumed;
-
-  public ReplicationStats(long sizeInBytes, long sizeInRecords, long entriesConsumed) {
-    this.sizeInBytes = sizeInBytes;
-    this.sizeInRecords = sizeInRecords;
-    this.entriesConsumed = entriesConsumed;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(sizeInBytes + sizeInRecords + entriesConsumed);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o != null) {
-      if (ReplicationStats.class.isAssignableFrom(o.getClass())) {
-        ReplicationStats other = (ReplicationStats) o;
-        return sizeInBytes == other.sizeInBytes && sizeInRecords == other.sizeInRecords
-            && entriesConsumed == other.entriesConsumed;
-      }
-    }
-    return false;
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
deleted file mode 100644
index f1d988f..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
+++ /dev/null
@@ -1,78 +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.tserver.replication;
-
-import java.util.concurrent.ThreadPoolExecutor;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Launches the {@link ReplicationProcessor}
- */
-@Deprecated
-public class ReplicationWorker implements Runnable {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationWorker.class);
-
-  private final ServerContext context;
-  private ThreadPoolExecutor executor;
-
-  public ReplicationWorker(ServerContext context) {
-    this.context = context;
-  }
-
-  public void setExecutor(ThreadPoolExecutor executor) {
-    this.executor = executor;
-  }
-
-  @Override
-  public void run() {
-    DefaultConfiguration defaultConf = DefaultConfiguration.getInstance();
-    long defaultDelay = defaultConf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_DELAY);
-    long defaultPeriod = defaultConf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_PERIOD);
-    AccumuloConfiguration conf = context.getConfiguration();
-    long delay = conf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_DELAY);
-    long period = conf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_PERIOD);
-    try {
-      DistributedWorkQueue workQueue;
-      if (defaultDelay != delay && defaultPeriod != period) {
-        log.debug("Configuration DistributedWorkQueue with delay and period of {} and {}", delay,
-            period);
-        workQueue = new DistributedWorkQueue(
-            context.getZooKeeperRoot() + ReplicationConstants.ZOO_WORK_QUEUE, conf, context, delay,
-            period);
-      } else {
-        log.debug("Configuring DistributedWorkQueue with default delay and period");
-        workQueue = new DistributedWorkQueue(
-            context.getZooKeeperRoot() + ReplicationConstants.ZOO_WORK_QUEUE, conf, context);
-      }
-
-      workQueue.startProcessing(new ReplicationProcessor(context), executor);
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/WalClientExecReturn.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/WalClientExecReturn.java
deleted file mode 100644
index eed0f3d..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/WalClientExecReturn.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.tserver.replication;
-
-import java.io.DataInputStream;
-import java.util.Set;
-
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.Exec;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-class WalClientExecReturn implements Exec<ReplicationStats,ReplicationServicer.Client> {
-
-  private static final Logger log = LoggerFactory.getLogger(WalClientExecReturn.class);
-
-  private final AccumuloReplicaSystem ars;
-  private ReplicationTarget target;
-  private DataInputStream input;
-  private Path p;
-  private Status status;
-  private long sizeLimit;
-  private String remoteTableId;
-  private TCredentials tcreds;
-  private Set<Integer> tids;
-
-  public WalClientExecReturn(AccumuloReplicaSystem ars, ReplicationTarget target,
-      DataInputStream input, Path p, Status status, long sizeLimit, String remoteTableId,
-      TCredentials tcreds, Set<Integer> tids) {
-    this.ars = ars;
-    this.target = target;
-    this.input = input;
-    this.p = p;
-    this.status = status;
-    this.sizeLimit = sizeLimit;
-    this.remoteTableId = remoteTableId;
-    this.tcreds = tcreds;
-    this.tids = tids;
-  }
-
-  @Override
-  public ReplicationStats execute(Client client) throws TException {
-    WalReplication edits = ars.getWalEdits(target, input, p, status, sizeLimit, tids);
-
-    log.debug(
-        "Read {} WAL entries and retained {} bytes of WAL entries for replication to peer '{}'",
-        (edits.entriesConsumed == Long.MAX_VALUE) ? "all remaining" : edits.entriesConsumed,
-        edits.sizeInBytes, p);
-
-    // If we have some edits to send
-    if (edits.walEdits.getEditsSize() > 0) {
-      log.debug("Sending {} edits", edits.walEdits.getEditsSize());
-      long entriesReplicated = client.replicateLog(remoteTableId, edits.walEdits, tcreds);
-      if (entriesReplicated == edits.numUpdates) {
-        log.debug("Replicated {} edits", entriesReplicated);
-      } else {
-        log.warn("Sent {} WAL entries for replication but {} were reported as replicated",
-            edits.numUpdates, entriesReplicated);
-      }
-
-      // We don't have to replicate every LogEvent in the file (only Mutation LogEvents), but we
-      // want to track progress in the file relative to all LogEvents (to avoid duplicative
-      // processing/replication)
-      return edits;
-    } else if (edits.entriesConsumed > 0) {
-      // Even if we send no data, we want to record a non-zero new begin value to avoid checking
-      // the same
-      // log entries multiple times to determine if they should be sent
-      return edits;
-    }
-
-    // No data sent (bytes nor records) and no progress made
-    return new ReplicationStats(0L, 0L, 0L);
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/WalReplication.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/WalReplication.java
deleted file mode 100644
index a41ad8c..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/WalReplication.java
+++ /dev/null
@@ -1,63 +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.tserver.replication;
-
-import java.util.Objects;
-
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-
-/**
- * A "struct" to avoid a nested Entry. Contains the resultant information from collecting data for
- * replication
- */
-@Deprecated
-public class WalReplication extends ReplicationStats {
-  /**
-   * The data to send over the wire
-   */
-  public WalEdits walEdits;
-
-  /**
-   * The number of updates contained in this batch
-   */
-  public long numUpdates;
-
-  public WalReplication(WalEdits edits, long size, long entriesConsumed, long numMutations) {
-    super(size, edits.getEditsSize(), entriesConsumed);
-    this.walEdits = edits;
-    this.numUpdates = numMutations;
-  }
-
-  @Override
-  public int hashCode() {
-    return super.hashCode() + Objects.hashCode(walEdits) + Objects.hashCode(numUpdates);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof WalReplication) {
-      WalReplication other = (WalReplication) o;
-
-      return super.equals(other) && walEdits.equals(other.walEdits)
-          && numUpdates == other.numUpdates;
-    }
-
-    return false;
-  }
-}
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
index 92bbcbc..caed908 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
@@ -19,10 +19,9 @@
 package org.apache.accumulo.tserver.session;
 
 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.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -45,9 +44,9 @@
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
-import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
-import org.apache.accumulo.core.tabletserver.thrift.ScanState;
-import org.apache.accumulo.core.tabletserver.thrift.ScanType;
+import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.ScanState;
+import org.apache.accumulo.core.tabletscan.thrift.ScanType;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.threads.ThreadPools;
@@ -65,18 +64,16 @@
 public class SessionManager {
   private static final Logger log = LoggerFactory.getLogger(SessionManager.class);
 
-  private static final SecureRandom random = new SecureRandom();
   private final ConcurrentMap<Long,Session> sessions = new ConcurrentHashMap<>();
   private final long maxIdle;
   private final long maxUpdateIdle;
   private final BlockingQueue<Session> deferredCleanupQueue = new ArrayBlockingQueue<>(5000);
   private final Long expiredSessionMarker = (long) -1;
-  private final AccumuloConfiguration aconf;
   private final ServerContext ctx;
 
   public SessionManager(ServerContext context) {
     this.ctx = context;
-    this.aconf = context.getConfiguration();
+    final AccumuloConfiguration aconf = context.getConfiguration();
     maxUpdateIdle = aconf.getTimeInMillis(Property.TSERV_UPDATE_SESSION_MAXIDLE);
     maxIdle = aconf.getTimeInMillis(Property.TSERV_SESSION_MAXIDLE);
 
@@ -87,7 +84,7 @@
   }
 
   public long createSession(Session session, boolean reserve) {
-    long sid = random.nextLong();
+    long sid = RANDOM.get().nextLong();
 
     synchronized (session) {
       Preconditions.checkArgument(session.state == State.NEW);
@@ -96,7 +93,7 @@
     }
 
     while (sessions.putIfAbsent(sid, session) != null) {
-      sid = random.nextLong();
+      sid = RANDOM.get().nextLong();
     }
 
     return sid;
@@ -231,29 +228,23 @@
       return true;
     }
 
-    boolean removed = false;
-
     synchronized (session) {
       if (session.state == State.RESERVED) {
         return false;
       }
-
       session.state = State.REMOVED;
-      removed = true;
     }
 
-    if (removed) {
-      sessions.remove(sessionId);
-    }
+    sessions.remove(sessionId);
 
-    return removed;
+    return true;
   }
 
   static void cleanup(BlockingQueue<Session> deferredCleanupQueue, Session session) {
     if (!session.cleanup()) {
-      var retry = Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS)
-          .incrementBy(25, MILLISECONDS).maxWait(5, SECONDS).backOffFactor(1.5)
-          .logInterval(1, MINUTES).createRetry();
+      var retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+          .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(5)).backOffFactor(1.5)
+          .logInterval(Duration.ofMinutes(1)).createRetry();
 
       while (!deferredCleanupQueue.offer(session)) {
         if (session.cleanup()) {
@@ -352,7 +343,7 @@
 
     Set<Entry<Long,Session>> copiedIdleSessions = new HashSet<>();
 
-    /**
+    /*
      * Add sessions so that get the list returned in the active scans call
      */
     for (Session session : deferredCleanupQueue) {
@@ -392,7 +383,7 @@
     final long ct = System.currentTimeMillis();
     final Set<Entry<Long,Session>> copiedIdleSessions = new HashSet<>();
 
-    /**
+    /*
      * Add sessions so that get the list returned in the active scans call
      */
     for (Session session : deferredCleanupQueue) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
index fc84c03..d519ffd 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
@@ -20,8 +20,6 @@
 
 import static org.apache.accumulo.tserver.TabletStatsKeeper.Operation.MAJOR;
 
-import java.io.IOException;
-import java.io.UncheckedIOException;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -52,10 +50,9 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.logging.TabletLogger;
-import org.apache.accumulo.core.manager.thrift.TabletLoadState;
 import org.apache.accumulo.core.metadata.CompactableFileImpl;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
@@ -69,15 +66,14 @@
 import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
 import org.apache.accumulo.core.util.compaction.CompactionJobImpl;
 import org.apache.accumulo.core.util.compaction.CompactionServicesConfig;
+import org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind;
 import org.apache.accumulo.core.util.ratelimit.RateLimiter;
 import org.apache.accumulo.server.ServiceEnvironmentImpl;
 import org.apache.accumulo.server.compaction.CompactionStats;
 import org.apache.accumulo.server.compaction.FileCompactor.CompactionCanceledException;
-import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.tserver.compactions.Compactable;
 import org.apache.accumulo.tserver.compactions.CompactionManager;
 import org.apache.accumulo.tserver.compactions.ExternalCompactionJob;
-import org.apache.accumulo.tserver.managermessage.TabletStatusMessage;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -114,10 +110,6 @@
 
   private final Set<CompactionServiceId> servicesUsed = new ConcurrentSkipListSet<>();
 
-  enum ChopSelectionStatus {
-    SELECTING, SELECTED, NOT_ACTIVE, MARKING
-  }
-
   // status of special compactions
   enum FileSelectionStatus {
     NEW, SELECTING, SELECTED, RESERVED, NOT_ACTIVE, CANCELED
@@ -144,9 +136,8 @@
   public interface CompactionHelper {
     Set<StoredTabletFile> selectFiles(SortedMap<StoredTabletFile,DataFileValue> allFiles);
 
-    Set<StoredTabletFile> getFilesToDrop();
-
-    Map<String,String> getConfigOverrides(Set<CompactableFile> files);
+    Map<String,String> getConfigOverrides(Set<CompactableFile> inputFiles,
+        Set<StoredTabletFile> selectedFiles, CompactionKind kind);
 
   }
 
@@ -170,7 +161,6 @@
    * <ul>
    * <li>Tracks which files are reserved for compactions
    * <li>Determines which files are available for compactions
-   * <li>Tracks which files are chopped and which need to be chopped
    * <li>Tracks which files are selected for user and selector compactions
    * <li>Coordinates the file selection process
    * </ul>
@@ -200,11 +190,6 @@
 
     protected Set<StoredTabletFile> allCompactingFiles = new HashSet<>();
 
-    // track files produced by compactions of this tablet, those are considered chopped
-    private final Set<StoredTabletFile> choppedFiles = new HashSet<>();
-    private ChopSelectionStatus chopStatus = ChopSelectionStatus.NOT_ACTIVE;
-    private final Set<StoredTabletFile> allFilesWhenChopStarted = new HashSet<>();
-
     private final KeyExtent extent;
     private final Deriver<Duration> selectionExpirationDeriver;
 
@@ -221,7 +206,7 @@
         this.selectStatus = FileSelectionStatus.RESERVED;
 
         log.debug("Selected compaction status initialized from external compactions {} {} {} {}",
-            getExtent(), selectStatus, initiallySelectedAll, asFileNames(selectedFiles));
+            getExtent(), selectStatus, initiallySelectedAll, asMinimalString(selectedFiles));
       }
     }
 
@@ -249,11 +234,13 @@
 
     boolean initiateSelection(CompactionKind kind) {
 
-      Preconditions.checkArgument(kind == CompactionKind.SELECTOR || kind == CompactionKind.USER);
+      Preconditions
+          .checkArgument(kind == DeprecatedCompactionKind.SELECTOR || kind == CompactionKind.USER);
 
-      if (selectStatus == FileSelectionStatus.NOT_ACTIVE || (kind == CompactionKind.USER
-          && selectKind == CompactionKind.SELECTOR && noneRunning(CompactionKind.SELECTOR)
-          && selectStatus != FileSelectionStatus.SELECTING)) {
+      if (selectStatus == FileSelectionStatus.NOT_ACTIVE
+          || (kind == CompactionKind.USER && selectKind == DeprecatedCompactionKind.SELECTOR
+              && noneRunning(DeprecatedCompactionKind.SELECTOR)
+              && selectStatus != FileSelectionStatus.SELECTING)) {
         selectStatus = FileSelectionStatus.NEW;
         selectKind = kind;
         selectedFiles.clear();
@@ -286,7 +273,7 @@
       selectedFiles.addAll(selected);
       initiallySelectedAll = allSelected;
       log.trace("Selected compaction status changed {} {} {} {}", getExtent(), selectStatus,
-          initiallySelectedAll, asFileNames(selectedFiles));
+          initiallySelectedAll, asMinimalString(selectedFiles));
       TabletLogger.selected(getExtent(), selectKind, selectedFiles);
     }
 
@@ -301,71 +288,6 @@
           || selectStatus == FileSelectionStatus.RESERVED) && kind == selectKind;
     }
 
-    ChopSelectionStatus getChopStatus() {
-      return chopStatus;
-    }
-
-    ChopSelector initiateChop(Set<StoredTabletFile> allFiles) {
-      Preconditions.checkState(chopStatus == ChopSelectionStatus.NOT_ACTIVE);
-      Set<StoredTabletFile> filesToExamine = new HashSet<>(allFiles);
-      chopStatus = ChopSelectionStatus.SELECTING;
-      filesToExamine.removeAll(choppedFiles);
-      filesToExamine.removeAll(allCompactingFiles);
-      return new ChopSelector(allFiles, filesToExamine);
-    }
-
-    class ChopSelector {
-      private final Set<StoredTabletFile> allFiles;
-      private final Set<StoredTabletFile> filesToExamine;
-
-      private ChopSelector(Set<StoredTabletFile> allFiles, Set<StoredTabletFile> filesToExamine) {
-        this.allFiles = allFiles;
-        this.filesToExamine = filesToExamine;
-      }
-
-      void selectChopFiles(Set<StoredTabletFile> unchoppedFiles) {
-        Preconditions.checkState(chopStatus == ChopSelectionStatus.SELECTING);
-        choppedFiles.addAll(Sets.difference(filesToExamine, unchoppedFiles));
-        chopStatus = ChopSelectionStatus.SELECTED;
-        allFilesWhenChopStarted.clear();
-        allFilesWhenChopStarted.addAll(allFiles);
-
-        var filesToChop = getFilesToChop(allFiles);
-        if (!filesToChop.isEmpty()) {
-          TabletLogger.selected(getExtent(), CompactionKind.CHOP, filesToChop);
-        }
-      }
-
-      Set<StoredTabletFile> getFilesToExamine() {
-        return Collections.unmodifiableSet(filesToExamine);
-      }
-    }
-
-    boolean finishChop(Set<StoredTabletFile> allFiles) {
-
-      boolean completed = false;
-
-      if (chopStatus == ChopSelectionStatus.SELECTED) {
-        if (getFilesToChop(allFiles).isEmpty()) {
-          chopStatus = ChopSelectionStatus.MARKING;
-          completed = true;
-        }
-      }
-
-      choppedFiles.retainAll(allFiles);
-
-      return completed;
-    }
-
-    void finishMarkingChop() {
-      Preconditions.checkState(chopStatus == ChopSelectionStatus.MARKING);
-      chopStatus = ChopSelectionStatus.NOT_ACTIVE;
-    }
-
-    void addChoppedFiles(Collection<StoredTabletFile> files) {
-      choppedFiles.addAll(files);
-    }
-
     void userCompactionCanceled() {
       if (isSelected(CompactionKind.USER)) {
         if (noneRunning(CompactionKind.USER)) {
@@ -378,19 +300,11 @@
       }
     }
 
-    private Set<StoredTabletFile> getFilesToChop(Set<StoredTabletFile> allFiles) {
-      Preconditions.checkState(chopStatus == ChopSelectionStatus.SELECTED);
-      var copy = new HashSet<>(allFilesWhenChopStarted);
-      copy.retainAll(allFiles);
-      copy.removeAll(choppedFiles);
-      return copy;
-    }
-
     /**
      * @return The set of tablet files that are candidates for compaction
      */
-    Set<StoredTabletFile> getCandidates(Set<StoredTabletFile> currFiles, CompactionKind kind,
-        boolean isCompactionStratConfigured) {
+    @SuppressWarnings("deprecation")
+    Set<StoredTabletFile> getCandidates(Set<StoredTabletFile> currFiles, CompactionKind kind) {
 
       if (!currFiles.containsAll(allCompactingFiles)) {
         log.trace("Ignoring because compacting not a subset {}", getExtent());
@@ -402,44 +316,12 @@
 
       switch (kind) {
         case SYSTEM: {
-          if (isCompactionStratConfigured) {
-            return Set.of();
-          }
-
           return handleSystemCompaction(currFiles);
         }
         case SELECTOR:
           // intentional fall through
         case USER:
           return handleUserSelectorCompaction(currFiles, kind);
-        case CHOP: {
-          return handleChopCompaction(currFiles);
-        }
-        default:
-          throw new AssertionError();
-      }
-    }
-
-    private Set<StoredTabletFile> handleChopCompaction(Set<StoredTabletFile> currFiles) {
-      switch (chopStatus) {
-        case NOT_ACTIVE:
-        case SELECTING:
-        case MARKING:
-          return Set.of();
-        case SELECTED: {
-          if (selectStatus == FileSelectionStatus.NEW
-              || selectStatus == FileSelectionStatus.SELECTING) {
-            return Set.of();
-          }
-
-          var filesToChop = getFilesToChop(currFiles);
-          filesToChop.removeAll(allCompactingFiles);
-          if (selectStatus == FileSelectionStatus.SELECTED
-              || selectStatus == FileSelectionStatus.RESERVED) {
-            filesToChop.removeAll(selectedFiles);
-          }
-          return Collections.unmodifiableSet(filesToChop);
-        }
         default:
           throw new AssertionError();
       }
@@ -534,11 +416,13 @@
           return false;
         case SELECTED:
         case RESERVED: {
-          if (job.getKind() == CompactionKind.USER || job.getKind() == CompactionKind.SELECTOR) {
+          if (job.getKind() == CompactionKind.USER
+              || job.getKind() == DeprecatedCompactionKind.SELECTOR) {
             if (selectKind == job.getKind()) {
               if (!selectedFiles.containsAll(jobFiles)) {
                 log.trace("Ignoring {} compaction that does not contain selected files {} {} {}",
-                    job.getKind(), getExtent(), asFileNames(selectedFiles), asFileNames(jobFiles));
+                    job.getKind(), getExtent(), asMinimalString(selectedFiles),
+                    asMinimalString(jobFiles));
                 return false;
               }
             } else {
@@ -548,14 +432,15 @@
             }
           } else if (!Collections.disjoint(selectedFiles, jobFiles)) {
             log.trace("Ingoing compaction that overlaps with selected files {} {} {}", getExtent(),
-                job.getKind(), asFileNames(Sets.intersection(selectedFiles, jobFiles)));
+                job.getKind(), asMinimalString(Sets.intersection(selectedFiles, jobFiles)));
             return false;
           }
           break;
         }
         case CANCELED:
         case NOT_ACTIVE: {
-          if (job.getKind() == CompactionKind.USER || job.getKind() == CompactionKind.SELECTOR) {
+          if (job.getKind() == CompactionKind.USER
+              || job.getKind() == DeprecatedCompactionKind.SELECTOR) {
             log.trace("Ignoring {} compaction because selectStatus is {} for {}", job.getKind(),
                 selectStatus, getExtent());
             return false;
@@ -591,20 +476,17 @@
         Optional<StoredTabletFile> newFile, boolean successful) {
       Preconditions.checkArgument(!jobFiles.isEmpty());
       Preconditions.checkState(allCompactingFiles.removeAll(jobFiles));
-      if (newFile.isPresent()) {
-        choppedFiles.add(newFile.orElseThrow());
-      }
 
-      if (successful
-          && (job.getKind() == CompactionKind.USER || job.getKind() == CompactionKind.SELECTOR)) {
+      if (successful && (job.getKind() == CompactionKind.USER
+          || job.getKind() == DeprecatedCompactionKind.SELECTOR)) {
         selectedCompactionCompleted(job, jobFiles, newFile);
       }
     }
 
     private void selectedCompactionCompleted(CompactionJob job, Set<StoredTabletFile> jobFiles,
         Optional<StoredTabletFile> newFile) {
-      Preconditions.checkArgument(
-          job.getKind() == CompactionKind.USER || job.getKind() == CompactionKind.SELECTOR);
+      Preconditions.checkArgument(job.getKind() == CompactionKind.USER
+          || job.getKind() == DeprecatedCompactionKind.SELECTOR);
       Preconditions.checkState(selectedFiles.containsAll(jobFiles));
       Preconditions.checkState((selectStatus == FileSelectionStatus.RESERVED
           || selectStatus == FileSelectionStatus.CANCELED) && selectKind == job.getKind());
@@ -620,7 +502,7 @@
           selectedFiles.add(newFile.orElseThrow());
         }
         log.trace("Compacted subset of selected files {} {} -> {}", getExtent(),
-            asFileNames(jobFiles), newFile.orElse(null));
+            asMinimalString(jobFiles), newFile.orElse(null));
       } else {
         log.debug("Canceled selected compaction completed {} but others still running ",
             getExtent());
@@ -661,7 +543,7 @@
             tabletCompactionId.get().orElseThrow().getSecond());
         this.compactionConfig = tabletCompactionId.get().orElseThrow().getSecond();
         this.compactionId = tabletCompactionId.get().orElseThrow().getFirst();
-      } else if (extSelInfo.orElseThrow().selectKind == CompactionKind.SELECTOR) {
+      } else if (extSelInfo.orElseThrow().selectKind == DeprecatedCompactionKind.SELECTOR) {
         this.chelper =
             CompactableUtils.getHelper(extSelInfo.orElseThrow().selectKind, tablet, null, null);
       }
@@ -701,11 +583,6 @@
       }
     });
 
-    if (extCompactions.values().stream().map(ecMeta -> ecMeta.getKind())
-        .anyMatch(kind -> kind == CompactionKind.CHOP)) {
-      initiateChop();
-    }
-
     this.servicesInUse = Suppliers.memoizeWithExpiration(() -> {
       HashSet<CompactionServiceId> servicesIds = new HashSet<>();
       for (CompactionKind kind : CompactionKind.values()) {
@@ -750,82 +627,10 @@
     return runningJobs.stream().noneMatch(job -> job.getKind() == kind);
   }
 
-  void initiateChop() {
-
-    Set<StoredTabletFile> allFiles = tablet.getDatafiles().keySet();
-    FileManager.ChopSelector chopSelector;
-
-    synchronized (this) {
-      if (fileMgr.getChopStatus() == ChopSelectionStatus.NOT_ACTIVE) {
-        chopSelector = fileMgr.initiateChop(allFiles);
-      } else {
-        return;
-      }
-    }
-
-    Set<StoredTabletFile> unchoppedFiles = selectChopFiles(chopSelector.getFilesToExamine());
-
-    synchronized (this) {
-      chopSelector.selectChopFiles(unchoppedFiles);
-    }
-
-    checkifChopComplete(tablet.getDatafiles().keySet());
-  }
-
-  private void checkifChopComplete(Set<StoredTabletFile> allFiles) {
-
-    boolean completed;
-
-    synchronized (this) {
-      if (closed) {
-        // if closed, do not attempt to transition to the MARKING state
-        return;
-      }
-      // when this returns true it means we transitioned to the MARKING state
-      completed = fileMgr.finishChop(allFiles);
-    }
-
-    if (completed) {
-      try {
-        markChopped();
-      } finally {
-        synchronized (this) {
-          // transition the state from MARKING to NOT_ACTIVE
-          fileMgr.finishMarkingChop();
-          this.notifyAll();
-        }
-      }
-
-      TabletLogger.selected(getExtent(), CompactionKind.CHOP, Set.of());
-    }
-  }
-
-  private void markChopped() {
-    MetadataTableUtil.chopped(tablet.getTabletServer().getContext(), getExtent(),
-        tablet.getTabletServer().getLock());
-    tablet.getTabletServer()
-        .enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.CHOPPED, getExtent()));
-  }
-
-  private Set<StoredTabletFile> selectChopFiles(Set<StoredTabletFile> chopCandidates) {
-    try {
-      var firstAndLastKeys = CompactableUtils.getFirstAndLastKeys(tablet, chopCandidates);
-      return CompactableUtils.findChopFiles(getExtent(), firstAndLastKeys, chopCandidates);
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
-
   /**
    * Tablet can use this to signal files were added.
    */
-  void filesAdded(boolean chopped, Collection<StoredTabletFile> files) {
-    if (chopped) {
-      synchronized (this) {
-        fileMgr.addChoppedFiles(files);
-      }
-    }
-
+  void filesAdded() {
     manager.compactableChanged(this);
   }
 
@@ -838,11 +643,11 @@
   }
 
   private void initiateSelection(CompactionKind kind) {
-    if (kind != CompactionKind.SELECTOR) {
+    if (kind != DeprecatedCompactionKind.SELECTOR) {
       return;
     }
 
-    initiateSelection(CompactionKind.SELECTOR, null, null);
+    initiateSelection(DeprecatedCompactionKind.SELECTOR, null, null);
   }
 
   private void checkIfUserCompactionCanceled() {
@@ -923,7 +728,8 @@
     for (Entry<ExternalCompactionId,ExternalCompactionMetadata> entry : extCompactions.entrySet()) {
       var ecMeta = entry.getValue();
 
-      if (ecMeta.getKind() != CompactionKind.USER && ecMeta.getKind() != CompactionKind.SELECTOR) {
+      if (ecMeta.getKind() != CompactionKind.USER
+          && ecMeta.getKind() != DeprecatedCompactionKind.SELECTOR) {
         continue;
       }
 
@@ -1002,7 +808,7 @@
       String reason = reasons.toString();
       extCompactions.entrySet().stream().filter(e -> {
         var kind = e.getValue().getKind();
-        return kind == CompactionKind.SELECTOR || kind == CompactionKind.USER;
+        return kind == DeprecatedCompactionKind.SELECTOR || kind == CompactionKind.USER;
       }).map(Entry::getKey).forEach(ecid -> externalCompactionsToRemove.putIfAbsent(ecid, reason));
       return Optional.empty();
     }
@@ -1016,7 +822,8 @@
 
   private void initiateSelection(CompactionKind kind, Long compactionId,
       CompactionConfig compactionConfig) {
-    Preconditions.checkArgument(kind == CompactionKind.USER || kind == CompactionKind.SELECTOR);
+    Preconditions
+        .checkArgument(kind == CompactionKind.USER || kind == DeprecatedCompactionKind.SELECTOR);
 
     var localHelper = CompactableUtils.getHelper(kind, tablet, compactionId, compactionConfig);
 
@@ -1074,8 +881,7 @@
           fileMgr.cancelSelection();
         }
       } else {
-        var allSelected =
-            allFiles.keySet().equals(Sets.union(selectingFiles, localHelper.getFilesToDrop()));
+        var allSelected = allFiles.keySet().equals(selectingFiles);
         synchronized (this) {
           fileMgr.finishSelection(selectingFiles, allSelected);
         }
@@ -1093,8 +899,8 @@
     }
   }
 
-  static Collection<String> asFileNames(Set<StoredTabletFile> files) {
-    return Collections2.transform(files, StoredTabletFile::getFileName);
+  static Collection<String> asMinimalString(Set<StoredTabletFile> files) {
+    return Collections2.transform(files, StoredTabletFile::toMinimalString);
   }
 
   @Override
@@ -1107,13 +913,6 @@
     return tablet.getExtent();
   }
 
-  @SuppressWarnings("removal")
-  private boolean isCompactionStratConfigured() {
-    var strategyClass = tablet.getTableConfiguration().get(Property.TABLE_COMPACTION_STRATEGY);
-    return tablet.getTableConfiguration().isPropertySet(Property.TABLE_COMPACTION_STRATEGY)
-        && strategyClass != null && !strategyClass.isBlank();
-  }
-
   @Override
   public Optional<Files> getFiles(CompactionServiceId service, CompactionKind kind) {
 
@@ -1140,8 +939,8 @@
 
       var runningJobsCopy = Set.copyOf(runningJobs);
 
-      Set<StoredTabletFile> candidates = fileMgr.getCandidates(
-          Collections.unmodifiableSet(files.keySet()), kind, isCompactionStratConfigured());
+      Set<StoredTabletFile> candidates =
+          fileMgr.getCandidates(Collections.unmodifiableSet(files.keySet()), kind);
 
       if (candidates.isEmpty()) {
         return Optional.empty();
@@ -1194,6 +993,7 @@
    * that things have changed and there is no longer anything to do for the job. In this case
    * Optional.empty() is returned.
    */
+  @SuppressWarnings("deprecation")
   private Optional<CompactionInfo> reserveFilesForCompaction(CompactionServiceId service,
       CompactionJob job) {
     CompactionInfo cInfo = new CompactionInfo();
@@ -1285,7 +1085,6 @@
       }
     }
 
-    checkifChopComplete(tablet.getDatafiles().keySet());
     selectFiles();
   }
 
@@ -1310,7 +1109,7 @@
       TabletLogger.compacting(getExtent(), job, cInfo.localCompactionCfg);
       tablet.incrementStatusMajor();
       var check = new CompactionCheck(service, kind, keepRunning, cInfo.checkCompactionId);
-      TabletFile tmpFileName = tablet.getNextMapFilenameForMajc(cInfo.propagateDeletes);
+      ReferencedTabletFile tmpFileName = tablet.getNextDataFilenameForMajc(cInfo.propagateDeletes);
       var compactEnv = new MajCEnv(kind, check, readLimiter, writeLimiter, cInfo.propagateDeletes);
 
       SortedMap<StoredTabletFile,DataFileValue> allFiles = tablet.getDatafiles();
@@ -1320,7 +1119,7 @@
       stats = CompactableUtils.compact(tablet, job, cInfo, compactEnv, compactFiles, tmpFileName);
 
       newFile = CompactableUtils.bringOnline(tablet.getDatafileManager(), cInfo, stats,
-          compactFiles, allFiles, kind, tmpFileName);
+          compactFiles, tmpFileName);
 
       TabletLogger.compacted(getExtent(), job, newFile.orElse(null));
       successful = true;
@@ -1328,10 +1127,11 @@
       log.debug("Compaction canceled {} ", getExtent());
     } catch (Exception e) {
       newFile = Optional.empty();
+      TabletLogger.compactionFailed(getExtent(), job, cInfo.localCompactionCfg);
       throw new RuntimeException(e);
     } finally {
       completeCompaction(job, cInfo.jobFiles, newFile, successful);
-      tablet.updateTimer(MAJOR, queuedTime, startTime, stats.getEntriesRead(), newFile == null);
+      tablet.updateTimer(MAJOR, queuedTime, startTime, stats.getEntriesRead(), newFile.isEmpty());
     }
   }
 
@@ -1349,10 +1149,11 @@
     var cInfo = ocInfo.orElseThrow();
 
     try {
-      Map<String,String> overrides =
-          CompactableUtils.getOverrides(job.getKind(), tablet, cInfo.localHelper, job.getFiles());
+      Map<String,String> overrides = CompactableUtils.getOverrides(job.getKind(), tablet,
+          cInfo.localHelper, job.getFiles(), cInfo.selectedFiles);
 
-      TabletFile compactTmpName = tablet.getNextMapFilenameForMajc(cInfo.propagateDeletes);
+      ReferencedTabletFile compactTmpName =
+          tablet.getNextDataFilenameForMajc(cInfo.propagateDeletes);
 
       ExternalCompactionInfo ecInfo = new ExternalCompactionInfo();
 
@@ -1380,6 +1181,8 @@
 
     } catch (Exception e) {
       externalCompactions.remove(externalCompactionId);
+      TabletLogger.externalCompactionFailed(getExtent(), externalCompactionId, job,
+          cInfo.localCompactionCfg);
       completeCompaction(job, cInfo.jobFiles, Optional.empty(), false);
       throw new RuntimeException(e);
     }
@@ -1576,19 +1379,18 @@
       closed = true;
 
       // Wait while internal jobs are running or external compactions are committing. When
-      // chopStatus is MARKING or selectStatus is SELECTING, there may be metadata table writes so
+      // selectStatus is SELECTING, there may be metadata table writes so
       // wait on those. Do not wait on external compactions that are running.
       Predicate<CompactionJob> jobsToWaitFor =
           job -> !((CompactionExecutorIdImpl) job.getExecutor()).isExternalId();
       while (runningJobs.stream().anyMatch(jobsToWaitFor)
           || !externalCompactionsCommitting.isEmpty()
-          || fileMgr.chopStatus == ChopSelectionStatus.MARKING
           || fileMgr.selectStatus == FileSelectionStatus.SELECTING) {
 
         log.debug(
-            "Closing {} is waiting on {} running compactions, {} committing external compactions, chop marking {}, file selection {}",
+            "Closing {} is waiting on {} running compactions, {} committing external compactions, file selection {}",
             getExtent(), runningJobs.stream().filter(jobsToWaitFor).count(),
-            externalCompactionsCommitting.size(), fileMgr.chopStatus == ChopSelectionStatus.MARKING,
+            externalCompactionsCommitting.size(),
             fileMgr.selectStatus == FileSelectionStatus.SELECTING);
 
         try {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
index e2aab70..0cff077 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
@@ -22,20 +22,17 @@
 import java.io.UncheckedIOException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
-import java.util.concurrent.ExecutionException;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
 import org.apache.accumulo.core.client.PluginEnvironment;
 import org.apache.accumulo.core.client.admin.CompactionConfig;
-import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
 import org.apache.accumulo.core.client.admin.PluginConfig;
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
 import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
@@ -44,7 +41,6 @@
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.client.summary.Summary;
-import org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil;
 import org.apache.accumulo.core.clientImpl.UserCompactionUtils;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -52,165 +48,42 @@
 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.TabletId;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.metadata.CompactableFileImpl;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.spi.cache.BlockCache;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.compaction.CompactionJob;
 import org.apache.accumulo.core.spi.compaction.CompactionKind;
-import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.summary.Gatherer;
 import org.apache.accumulo.core.summary.SummarizerFactory;
 import org.apache.accumulo.core.summary.SummaryCollection;
 import org.apache.accumulo.core.summary.SummaryReader;
-import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind;
 import org.apache.accumulo.server.ServiceEnvironmentImpl;
 import org.apache.accumulo.server.compaction.CompactionStats;
 import org.apache.accumulo.server.compaction.FileCompactor;
 import org.apache.accumulo.server.compaction.FileCompactor.CompactionCanceledException;
 import org.apache.accumulo.server.compaction.FileCompactor.CompactionEnv;
 import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.CompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-import org.apache.accumulo.tserver.compaction.WriteParameters;
 import org.apache.accumulo.tserver.tablet.CompactableImpl.CompactionHelper;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Collections2;
 
-@SuppressWarnings("removal")
 public class CompactableUtils {
 
-  private static final Logger log = LoggerFactory.getLogger(CompactableUtils.class);
-
-  private final static Cache<TableId,Boolean> strategyWarningsCache =
-      CacheBuilder.newBuilder().maximumSize(1000).build();
-
-  public static Map<StoredTabletFile,Pair<Key,Key>> getFirstAndLastKeys(Tablet tablet,
-      Set<StoredTabletFile> allFiles) throws IOException {
-    final Map<StoredTabletFile,Pair<Key,Key>> result = new HashMap<>();
-    final FileOperations fileFactory = FileOperations.getInstance();
-    final VolumeManager fs = tablet.getTabletServer().getVolumeManager();
-    final TableConfiguration tableConf = tablet.getTableConfiguration();
-    final CryptoService cs = tableConf.getCryptoService();
-    for (StoredTabletFile file : allFiles) {
-      FileSystem ns = fs.getFileSystemByPath(file.getPath());
-      try (FileSKVIterator openReader =
-          fileFactory.newReaderBuilder().forFile(file.getPathStr(), ns, ns.getConf(), cs)
-              .withTableConfiguration(tableConf).seekToBeginning().build()) {
-        Key first = openReader.getFirstKey();
-        Key last = openReader.getLastKey();
-        result.put(file, new Pair<>(first, last));
-      }
-    }
-    return result;
-  }
-
-  public static Set<StoredTabletFile> findChopFiles(KeyExtent extent,
-      Map<StoredTabletFile,Pair<Key,Key>> firstAndLastKeys, Collection<StoredTabletFile> allFiles) {
-    Set<StoredTabletFile> result = new HashSet<>();
-
-    for (StoredTabletFile file : allFiles) {
-      Pair<Key,Key> pair = firstAndLastKeys.get(file);
-      Key first = pair.getFirst();
-      Key last = pair.getSecond();
-      // If first and last are null, it's an empty file. Add it to the compact set so it goes
-      // away.
-      if ((first == null && last == null) || (first != null && !extent.contains(first.getRow()))
-          || (last != null && !extent.contains(last.getRow()))) {
-        result.add(file);
-      }
-
-    }
-    return result;
-  }
-
-  static CompactionPlan selectFiles(CompactionKind kind, Tablet tablet,
-      SortedMap<StoredTabletFile,DataFileValue> datafiles, CompactionStrategyConfig csc) {
-
-    var trsm = tablet.getTabletResources().getTabletServerResourceManager();
-
-    BlockCache sc = trsm.getSummaryCache();
-    BlockCache ic = trsm.getIndexCache();
-    Cache<String,Long> fileLenCache = trsm.getFileLenCache();
-    MajorCompactionRequest request = new MajorCompactionRequest(tablet.getExtent(),
-        CompactableUtils.from(kind), tablet.getTabletServer().getVolumeManager(),
-        tablet.getTableConfiguration(), sc, ic, fileLenCache, tablet.getContext());
-
-    request.setFiles(datafiles);
-
-    CompactionStrategy strategy = CompactableUtils.newInstance(tablet.getTableConfiguration(),
-        csc.getClassName(), CompactionStrategy.class);
-    strategy.init(csc.getOptions());
-
-    try {
-      if (strategy.shouldCompact(request)) {
-        strategy.gatherInformation(request);
-        var plan = strategy.getCompactionPlan(request);
-
-        if (plan == null) {
-          return new CompactionPlan();
-        }
-
-        log.debug("Selected files using compaction strategy {} {} {} {}",
-            strategy.getClass().getSimpleName(), csc.getOptions(), plan.inputFiles,
-            plan.deleteFiles);
-
-        plan.validate(datafiles.keySet());
-
-        return plan;
-      }
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-    return new CompactionPlan();
-  }
-
-  static Map<String,String> computeOverrides(WriteParameters p) {
-    if (p == null) {
-      return null;
-    }
-
-    Map<String,String> result = new HashMap<>();
-    if (p.getHdfsBlockSize() > 0) {
-      result.put(Property.TABLE_FILE_BLOCK_SIZE.getKey(), "" + p.getHdfsBlockSize());
-    }
-    if (p.getBlockSize() > 0) {
-      result.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey(), "" + p.getBlockSize());
-    }
-    if (p.getIndexBlockSize() > 0) {
-      result.put(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey(),
-          "" + p.getIndexBlockSize());
-    }
-    if (p.getCompressType() != null) {
-      result.put(Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), p.getCompressType());
-    }
-    if (p.getReplication() != 0) {
-      result.put(Property.TABLE_FILE_REPLICATION.getKey(), "" + p.getReplication());
-    }
-    return result;
-  }
-
-  static Map<String,String> computeOverrides(Tablet tablet, Set<CompactableFile> files) {
+  static Map<String,String> computeOverrides(Tablet tablet, Set<CompactableFile> inputFiles,
+      Set<StoredTabletFile> selectedFiles, CompactionKind kind) {
     var tconf = tablet.getTableConfiguration();
 
     var configurorClass = tconf.get(Property.TABLE_COMPACTION_CONFIGURER);
@@ -220,11 +93,12 @@
 
     var opts = tconf.getAllPropertiesWithPrefixStripped(Property.TABLE_COMPACTION_CONFIGURER_OPTS);
 
-    return computeOverrides(tablet, files, new PluginConfig(configurorClass, opts));
+    return computeOverrides(tablet, inputFiles, selectedFiles,
+        new PluginConfig(configurorClass, opts), kind);
   }
 
-  static Map<String,String> computeOverrides(Tablet tablet, Set<CompactableFile> files,
-      PluginConfig cfg) {
+  static Map<String,String> computeOverrides(Tablet tablet, Set<CompactableFile> inputFiles,
+      Set<StoredTabletFile> selectedFiles, PluginConfig cfg, CompactionKind kind) {
     CompactionConfigurer configurer = CompactableUtils.newInstance(tablet.getTableConfiguration(),
         cfg.getClassName(), CompactionConfigurer.class);
 
@@ -250,7 +124,18 @@
     var overrides = configurer.override(new CompactionConfigurer.InputParameters() {
       @Override
       public Collection<CompactableFile> getInputFiles() {
-        return files;
+        return inputFiles;
+      }
+
+      @Override
+      public Set<CompactableFile> getSelectedFiles() {
+        if (kind == CompactionKind.USER || kind == DeprecatedCompactionKind.SELECTOR) {
+          var dataFileSizes = tablet.getDatafileManager().getDatafileSizes();
+          return selectedFiles.stream().map(f -> new CompactableFileImpl(f, dataFileSizes.get(f)))
+              .collect(Collectors.toSet());
+        } else { // kind == CompactionKind.SYSTEM
+          return Collections.emptySet();
+        }
       }
 
       @Override
@@ -262,6 +147,11 @@
       public TableId getTableId() {
         return tablet.getExtent().tableId();
       }
+
+      @Override
+      public TabletId getTabletId() {
+        return new TabletIdImpl(tablet.getExtent());
+      }
     });
 
     if (overrides.getOverrides().isEmpty()) {
@@ -276,8 +166,8 @@
     String context = ClassLoaderUtil.tableContext(tableConfig);
     try {
       return ConfigurationTypeHelper.getClassInstance(context, className, baseClass);
-    } catch (IOException | ReflectiveOperationException e) {
-      throw new RuntimeException(e);
+    } catch (ReflectiveOperationException e) {
+      throw new IllegalArgumentException(e);
     }
   }
 
@@ -348,6 +238,11 @@
       }
 
       @Override
+      public TabletId getTabletId() {
+        return new TabletIdImpl(tablet.getExtent());
+      }
+
+      @Override
       public Optional<SortedKeyValueIterator<Key,Value>> getSample(CompactableFile file,
           SamplerConfiguration sc) {
         try {
@@ -356,7 +251,8 @@
           FileSystem ns = tablet.getTabletServer().getVolumeManager().getFileSystemByPath(path);
           var tableConf = tablet.getTableConfiguration();
           var fiter = fileFactory.newReaderBuilder()
-              .forFile(path.toString(), ns, ns.getConf(), tableConf.getCryptoService())
+              .forFile(ReferencedTabletFile.of(path), ns, ns.getConf(),
+                  tableConf.getCryptoService())
               .withTableConfiguration(tableConf).seekToBeginning().build();
           return Optional.ofNullable(fiter.getSample(new SamplerConfigurationImpl(sc)));
         } catch (IOException e) {
@@ -371,41 +267,22 @@
 
   private static final class TableCompactionHelper implements CompactionHelper {
     private final PluginConfig cselCfg2;
-    private final CompactionStrategyConfig stratCfg2;
     private final Tablet tablet;
-    private WriteParameters wp;
-    private Set<StoredTabletFile> filesToDrop;
 
-    private TableCompactionHelper(PluginConfig cselCfg2, CompactionStrategyConfig stratCfg2,
-        Tablet tablet) {
-      this.cselCfg2 = cselCfg2;
-      this.stratCfg2 = stratCfg2;
-      this.tablet = tablet;
+    private TableCompactionHelper(PluginConfig cselCfg2, Tablet tablet) {
+      this.cselCfg2 = Objects.requireNonNull(cselCfg2);
+      this.tablet = Objects.requireNonNull(tablet);
     }
 
     @Override
     public Set<StoredTabletFile> selectFiles(SortedMap<StoredTabletFile,DataFileValue> allFiles) {
-      if (cselCfg2 != null) {
-        filesToDrop = Set.of();
-        return CompactableUtils.selectFiles(tablet, allFiles, cselCfg2);
-      } else {
-        var plan =
-            CompactableUtils.selectFiles(CompactionKind.SELECTOR, tablet, allFiles, stratCfg2);
-        this.wp = plan.writeParameters;
-        filesToDrop = Set.copyOf(plan.deleteFiles);
-        return Set.copyOf(plan.inputFiles);
-      }
+      return CompactableUtils.selectFiles(tablet, allFiles, cselCfg2);
     }
 
     @Override
-    public Map<String,String> getConfigOverrides(Set<CompactableFile> files) {
-      return computeOverrides(wp);
-    }
-
-    @Override
-    public Set<StoredTabletFile> getFilesToDrop() {
-      Preconditions.checkState(filesToDrop != null);
-      return filesToDrop;
+    public Map<String,String> getConfigOverrides(Set<CompactableFile> inputFiles,
+        Set<StoredTabletFile> selectedFiles, CompactionKind kind) {
+      return null;
     }
   }
 
@@ -413,8 +290,6 @@
     private final CompactionConfig compactionConfig;
     private final Tablet tablet;
     private final Long compactionId;
-    private WriteParameters wp;
-    private Set<StoredTabletFile> filesToDrop;
 
     private UserCompactionHelper(CompactionConfig compactionConfig, Tablet tablet,
         Long compactionId) {
@@ -426,21 +301,13 @@
     @Override
     public Set<StoredTabletFile> selectFiles(SortedMap<StoredTabletFile,DataFileValue> allFiles) {
 
-      Set<StoredTabletFile> selectedFiles;
+      final Set<StoredTabletFile> selectedFiles;
 
-      if (!CompactionStrategyConfigUtil.isDefault(compactionConfig.getCompactionStrategy())) {
-        var plan = CompactableUtils.selectFiles(CompactionKind.USER, tablet, allFiles,
-            compactionConfig.getCompactionStrategy());
-        this.wp = plan.writeParameters;
-        selectedFiles = Set.copyOf(plan.inputFiles);
-        filesToDrop = Set.copyOf(plan.deleteFiles);
-      } else if (!UserCompactionUtils.isDefault(compactionConfig.getSelector())) {
+      if (!UserCompactionUtils.isDefault(compactionConfig.getSelector())) {
         selectedFiles =
             CompactableUtils.selectFiles(tablet, allFiles, compactionConfig.getSelector());
-        filesToDrop = Set.of();
       } else {
         selectedFiles = allFiles.keySet();
-        filesToDrop = Set.of();
       }
 
       if (selectedFiles.isEmpty()) {
@@ -454,66 +321,39 @@
     }
 
     @Override
-    public Map<String,String> getConfigOverrides(Set<CompactableFile> files) {
+    public Map<String,String> getConfigOverrides(Set<CompactableFile> inputFiles,
+        Set<StoredTabletFile> selectedFiles, CompactionKind kind) {
       if (!UserCompactionUtils.isDefault(compactionConfig.getConfigurer())) {
-        return computeOverrides(tablet, files, compactionConfig.getConfigurer());
-      } else if (!CompactionStrategyConfigUtil.isDefault(compactionConfig.getCompactionStrategy())
-          && wp != null) {
-        return computeOverrides(wp);
+        return computeOverrides(tablet, inputFiles, selectedFiles, compactionConfig.getConfigurer(),
+            kind);
       }
 
       return null;
     }
-
-    @Override
-    public Set<StoredTabletFile> getFilesToDrop() {
-      Preconditions.checkState(filesToDrop != null);
-      return filesToDrop;
-    }
   }
 
+  @SuppressWarnings("deprecation")
+  private static final Property SELECTOR_PROP = Property.TABLE_COMPACTION_SELECTOR;
+  @SuppressWarnings("deprecation")
+  private static final Property SELECTOR_OPTS_PROP = Property.TABLE_COMPACTION_SELECTOR_OPTS;
+
   public static CompactionHelper getHelper(CompactionKind kind, Tablet tablet, Long compactionId,
       CompactionConfig compactionConfig) {
     if (kind == CompactionKind.USER) {
       return new UserCompactionHelper(compactionConfig, tablet, compactionId);
-    } else if (kind == CompactionKind.SELECTOR) {
+    } else if (kind == DeprecatedCompactionKind.SELECTOR) {
       var tconf = tablet.getTableConfiguration();
-      var selectorClassName = tconf.get(Property.TABLE_COMPACTION_SELECTOR);
+      var selectorClassName = tconf.get(SELECTOR_PROP);
 
       PluginConfig cselCfg = null;
 
       if (selectorClassName != null && !selectorClassName.isBlank()) {
-        var opts =
-            tconf.getAllPropertiesWithPrefixStripped(Property.TABLE_COMPACTION_SELECTOR_OPTS);
+        var opts = tconf.getAllPropertiesWithPrefixStripped(SELECTOR_OPTS_PROP);
         cselCfg = new PluginConfig(selectorClassName, opts);
       }
 
-      CompactionStrategyConfig stratCfg = null;
-
-      var stratClassName = tconf.get(Property.TABLE_COMPACTION_STRATEGY);
-      if (cselCfg == null && tconf.isPropertySet(Property.TABLE_COMPACTION_STRATEGY)
-          && stratClassName != null && !stratClassName.isBlank()) {
-        try {
-          strategyWarningsCache.get(tablet.getExtent().tableId(), () -> {
-            log.warn(
-                "Table id {} set {} to {}.  Compaction strategies are deprecated.  See the Javadoc"
-                    + " for class {} for more details.",
-                tablet.getExtent().tableId(), Property.TABLE_COMPACTION_STRATEGY.getKey(),
-                stratClassName, CompactionStrategyConfig.class.getName());
-            return true;
-          });
-        } catch (ExecutionException e) {
-          throw new RuntimeException(e);
-        }
-
-        var opts =
-            tconf.getAllPropertiesWithPrefixStripped(Property.TABLE_COMPACTION_STRATEGY_PREFIX);
-
-        stratCfg = new CompactionStrategyConfig(stratClassName).setOptions(opts);
-      }
-
-      if (cselCfg != null || stratCfg != null) {
-        return new TableCompactionHelper(cselCfg, stratCfg, tablet);
+      if (cselCfg != null) {
+        return new TableCompactionHelper(cselCfg, tablet);
       }
     }
 
@@ -521,16 +361,17 @@
   }
 
   public static Map<String,String> getOverrides(CompactionKind kind, Tablet tablet,
-      CompactionHelper driver, Set<CompactableFile> files) {
+      CompactionHelper driver, Set<CompactableFile> inputFiles,
+      Set<StoredTabletFile> selectedFiles) {
 
     Map<String,String> overrides = null;
 
-    if (kind == CompactionKind.USER || kind == CompactionKind.SELECTOR) {
-      overrides = driver.getConfigOverrides(files);
+    if (kind == CompactionKind.USER || kind == DeprecatedCompactionKind.SELECTOR) {
+      overrides = driver.getConfigOverrides(inputFiles, selectedFiles, kind);
     }
 
     if (overrides == null) {
-      overrides = computeOverrides(tablet, files);
+      overrides = computeOverrides(tablet, inputFiles, selectedFiles, kind);
     }
 
     if (overrides == null) {
@@ -557,16 +398,17 @@
    */
   static CompactionStats compact(Tablet tablet, CompactionJob job,
       CompactableImpl.CompactionInfo cInfo, CompactionEnv cenv,
-      Map<StoredTabletFile,DataFileValue> compactFiles, TabletFile tmpFileName)
+      Map<StoredTabletFile,DataFileValue> compactFiles, ReferencedTabletFile tmpFileName)
       throws IOException, CompactionCanceledException {
     TableConfiguration tableConf = tablet.getTableConfiguration();
 
-    AccumuloConfiguration compactionConfig = getCompactionConfig(tableConf,
-        getOverrides(job.getKind(), tablet, cInfo.localHelper, job.getFiles()));
+    AccumuloConfiguration compactionConfig =
+        getCompactionConfig(tableConf, getOverrides(job.getKind(), tablet, cInfo.localHelper,
+            job.getFiles(), cInfo.selectedFiles));
 
     FileCompactor compactor = new FileCompactor(tablet.getContext(), tablet.getExtent(),
         compactFiles, tmpFileName, cInfo.propagateDeletes, cenv, cInfo.iters, compactionConfig,
-        tableConf.getCryptoService());
+        tableConf.getCryptoService(), tablet.getPausedCompactionMetrics());
 
     return compactor.call();
   }
@@ -576,45 +418,24 @@
    */
   static Optional<StoredTabletFile> bringOnline(DatafileManager datafileManager,
       CompactableImpl.CompactionInfo cInfo, CompactionStats stats,
-      Map<StoredTabletFile,DataFileValue> compactFiles,
-      SortedMap<StoredTabletFile,DataFileValue> allFiles, CompactionKind kind,
-      TabletFile compactTmpName) throws IOException {
-    if (kind == CompactionKind.USER || kind == CompactionKind.SELECTOR) {
-      cInfo.localHelper.getFilesToDrop().forEach(f -> {
-        if (allFiles.containsKey(f)) {
-          compactFiles.put(f, allFiles.get(f));
-        }
-      });
-    }
+      Map<StoredTabletFile,DataFileValue> compactFiles, ReferencedTabletFile compactTmpName)
+      throws IOException {
+
     var dfv = new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
     return datafileManager.bringMajorCompactionOnline(compactFiles.keySet(), compactTmpName,
         cInfo.checkCompactionId, cInfo.selectedFiles, dfv, Optional.empty());
   }
 
-  public static MajorCompactionReason from(CompactionKind ck) {
-    switch (ck) {
-      case CHOP:
-        return MajorCompactionReason.CHOP;
-      case SYSTEM:
-      case SELECTOR:
-        return MajorCompactionReason.NORMAL;
-      case USER:
-        return MajorCompactionReason.USER;
-      default:
-        throw new IllegalArgumentException("Unknown kind " + ck);
-    }
-  }
-
-  public static TabletFile computeCompactionFileDest(TabletFile tmpFile) {
-    String newFilePath = tmpFile.getMetaInsert();
+  public static ReferencedTabletFile computeCompactionFileDest(ReferencedTabletFile tmpFile) {
+    String newFilePath = tmpFile.getNormalizedPathStr();
     int idx = newFilePath.indexOf("_tmp");
     if (idx > 0) {
       newFilePath = newFilePath.substring(0, idx);
     } else {
-      throw new IllegalArgumentException(
-          "Expected compaction tmp file " + tmpFile.getMetaInsert() + " to have suffix '_tmp'");
+      throw new IllegalArgumentException("Expected compaction tmp file "
+          + tmpFile.getNormalizedPathStr() + " to have suffix '_tmp'");
     }
-    return new TabletFile(new Path(newFilePath));
+    return new ReferencedTabletFile(new Path(newFilePath));
   }
 
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index b2568f7..1c534de 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -38,19 +38,18 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.logging.TabletLogger;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -86,7 +85,7 @@
         new AtomicReference<>(new MetadataUpdateCount(tablet.getExtent(), 0L, 0L));
   }
 
-  private final Set<TabletFile> filesToDeleteAfterScan = new HashSet<>();
+  private final Set<StoredTabletFile> filesToDeleteAfterScan = new HashSet<>();
   private final Map<Long,Set<StoredTabletFile>> scanFileReservations = new HashMap<>();
   private final MapCounter<StoredTabletFile> fileScanReferenceCounts = new MapCounter<>();
   private long nextScanReservationId = 0;
@@ -97,7 +96,7 @@
     }
   }
 
-  Pair<Long,Map<TabletFile,DataFileValue>> reserveFilesForScan() {
+  Pair<Long,Map<StoredTabletFile,DataFileValue>> reserveFilesForScan() {
     synchronized (tablet) {
 
       Set<StoredTabletFile> absFilePaths = new HashSet<>(datafileSizes.keySet());
@@ -106,7 +105,7 @@
 
       scanFileReservations.put(rid, absFilePaths);
 
-      Map<TabletFile,DataFileValue> ret = new HashMap<>();
+      Map<StoredTabletFile,DataFileValue> ret = new HashMap<>();
 
       for (StoredTabletFile path : absFilePaths) {
         fileScanReferenceCounts.increment(path, 1);
@@ -218,14 +217,14 @@
     return inUse;
   }
 
-  public Collection<StoredTabletFile> importMapFiles(long tid, Map<TabletFile,DataFileValue> paths,
-      boolean setTime) throws IOException {
+  public Collection<StoredTabletFile> importDataFiles(long tid,
+      Map<ReferencedTabletFile,DataFileValue> paths, boolean setTime) throws IOException {
 
     String bulkDir = null;
     // once tablet files are inserted into the metadata they will become StoredTabletFiles
     Map<StoredTabletFile,DataFileValue> newFiles = new HashMap<>(paths.size());
 
-    for (TabletFile tpath : paths.keySet()) {
+    for (ReferencedTabletFile tpath : paths.keySet()) {
       boolean inTheRightDirectory = false;
       Path parent = tpath.getPath().getParent().getParent();
       for (String tablesDir : tablet.getContext().getTablesDirs()) {
@@ -281,7 +280,7 @@
           datafileSizes.put(tpath.getKey(), tpath.getValue());
         }
 
-        tablet.getTabletResources().importedMapFiles();
+        tablet.getTabletResources().importedDataFiles();
 
         tablet.computeNumEntries();
       }
@@ -308,8 +307,9 @@
    * entries so was not inserted into the metadata. In this case empty is returned. If the file was
    * stored in the metadata table, then StoredTableFile will be returned.
    */
-  Optional<StoredTabletFile> bringMinorCompactionOnline(TabletFile tmpDatafile,
-      TabletFile newDatafile, DataFileValue dfv, CommitSession commitSession, long flushId) {
+  Optional<StoredTabletFile> bringMinorCompactionOnline(ReferencedTabletFile tmpDatafile,
+      ReferencedTabletFile newDatafile, DataFileValue dfv, CommitSession commitSession,
+      long flushId) {
     Optional<StoredTabletFile> newFile;
     // rename before putting in metadata table, so files in metadata table should
     // always exist
@@ -322,7 +322,7 @@
           vm.deleteRecursively(tmpDatafile.getPath());
         } else {
           if (!attemptedRename && vm.exists(newDatafile.getPath())) {
-            log.warn("Target map file already exist {}", newDatafile);
+            log.warn("Target data file already exist {}", newDatafile);
             throw new RuntimeException("File unexpectedly exists " + newDatafile.getPath());
           }
           // the following checks for spurious rename failures that succeeded but gave an IoE
@@ -356,7 +356,7 @@
         // The following call pairs with tablet.finishClearingUnusedLogs() later in this block. If
         // moving where the following method is called, examine it and finishClearingUnusedLogs()
         // before moving.
-        Set<String> unusedWalLogs = tablet.beginClearingUnusedLogs();
+        Set<LogEntry> unusedWalLogs = tablet.beginClearingUnusedLogs();
 
         // the order of writing to metadata and walog is important in the face of machine/process
         // failures need to write to metadata before writing to walog, when things are done in the
@@ -365,44 +365,6 @@
         newFile = tablet.updateTabletDataFile(commitSession.getMaxCommittedTime(), newDatafile, dfv,
             unusedWalLogs, flushId);
 
-        // Mark that we have data we want to replicate
-        // This WAL could still be in use by other Tablets *from the same table*, so we can only
-        // mark
-        // that there is data to replicate,
-        // but it is *not* closed. We know it is not closed by the fact that this MinC triggered. A
-        // MinC cannot happen unless the
-        // tablet is online and thus these WALs are referenced by that tablet. Therefore, the WAL
-        // replication status cannot be 'closed'.
-        @SuppressWarnings("deprecation")
-        boolean replicate = org.apache.accumulo.core.replication.ReplicationConfigurationUtil
-            .isEnabled(tablet.getExtent(), tablet.getTableConfiguration());
-        if (replicate) {
-          // unusedWalLogs is of the form host/fileURI, need to strip off the host portion
-          Set<String> logFileOnly = new HashSet<>();
-          for (String unusedWalLog : unusedWalLogs) {
-            int index = unusedWalLog.indexOf('/');
-            if (index == -1) {
-              log.warn(
-                  "Could not find host component to strip from DFSLogger representation of WAL");
-            } else {
-              unusedWalLog = unusedWalLog.substring(index + 1);
-            }
-            logFileOnly.add(unusedWalLog);
-          }
-
-          if (log.isDebugEnabled()) {
-            log.debug("Recording that data has been ingested into {} using {}", tablet.getExtent(),
-                logFileOnly);
-          }
-          for (String logFile : logFileOnly) {
-            @SuppressWarnings("deprecation")
-            Status status =
-                org.apache.accumulo.server.replication.StatusUtil.openWithUnknownLength();
-            ReplicationTableUtil.updateFiles(tablet.getContext(), tablet.getExtent(), logFile,
-                status);
-          }
-        }
-
         tablet.finishClearingUnusedLogs();
       } finally {
         tablet.getLogLock().unlock();
@@ -464,17 +426,17 @@
   }
 
   Optional<StoredTabletFile> bringMajorCompactionOnline(Set<StoredTabletFile> oldDatafiles,
-      TabletFile tmpDatafile, Long compactionId, Set<StoredTabletFile> selectedFiles,
+      ReferencedTabletFile tmpDatafile, Long compactionId, Set<StoredTabletFile> selectedFiles,
       DataFileValue dfv, Optional<ExternalCompactionId> ecid) throws IOException {
     final KeyExtent extent = tablet.getExtent();
     VolumeManager vm = tablet.getTabletServer().getContext().getVolumeManager();
     long t1, t2;
 
-    TabletFile newDatafile = CompactableUtils.computeCompactionFileDest(tmpDatafile);
+    ReferencedTabletFile newDatafile = CompactableUtils.computeCompactionFileDest(tmpDatafile);
 
     if (vm.exists(newDatafile.getPath())) {
-      log.error("Target map file already exist " + newDatafile, new Exception());
-      throw new IllegalStateException("Target map file already exist " + newDatafile);
+      log.error("Target data file already exist " + newDatafile, new Exception());
+      throw new IllegalStateException("Target data file already exist " + newDatafile);
     }
 
     if (dfv.getNumEntries() == 0) {
@@ -571,9 +533,9 @@
     }
   }
 
-  public Set<TabletFile> getFiles() {
+  public Set<StoredTabletFile> getFiles() {
     synchronized (tablet) {
-      HashSet<TabletFile> files = new HashSet<>(datafileSizes.keySet());
+      HashSet<StoredTabletFile> files = new HashSet<>(datafileSizes.keySet());
       return Collections.unmodifiableSet(files);
     }
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MajCEnv.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MajCEnv.java
index f878715..e81f98d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MajCEnv.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MajCEnv.java
@@ -81,16 +81,16 @@
   }
 
   @Override
+  @SuppressWarnings("deprecation")
   public TCompactionReason getReason() {
     switch (kind) {
       case USER:
         return TCompactionReason.USER;
-      case CHOP:
-        return TCompactionReason.CHOP;
       case SELECTOR:
       case SYSTEM:
-      default:
         return TCompactionReason.SYSTEM;
+      default:
+        throw new IllegalStateException("Unknown compaction kind " + kind);
     }
   }
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
index f190d79..d978ed8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
@@ -18,7 +18,8 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.file.FilePrefix;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.tserver.MinorCompactionReason;
@@ -67,15 +68,16 @@
         } finally {
           span2.end();
         }
-        TabletFile newFile = null;
-        TabletFile tmpFile = null;
+        ReferencedTabletFile newFile = null;
+        ReferencedTabletFile tmpFile = null;
         Span span3 = TraceUtil.startSpan(this.getClass(), "start");
         try (Scope scope3 = span3.makeCurrent()) {
           while (true) {
             try {
               if (newFile == null) {
-                newFile = tablet.getNextMapFilename("F");
-                tmpFile = new TabletFile(new Path(newFile.getPathStr() + "_tmp"));
+                newFile = tablet.getNextDataFilename(FilePrefix.MINOR_COMPACTION);
+                tmpFile =
+                    new ReferencedTabletFile(new Path(newFile.getNormalizedPathStr() + "_tmp"));
               }
               /*
                * the purpose of the minor compaction start event is to keep track of the filename...
@@ -86,7 +88,7 @@
                * for the minor compaction
                */
               tablet.getTabletServer().minorCompactionStarted(commitSession,
-                  commitSession.getWALogSeq() + 1, newFile.getMetaInsert());
+                  commitSession.getWALogSeq() + 1, newFile.insert().getMetadataPath());
               break;
             } catch (Exception e) {
               // Catching Exception here rather than something more specific as we can't allow the
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
index 3e5be5d..2a3dc01 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
@@ -30,7 +30,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.server.compaction.CompactionStats;
 import org.apache.accumulo.server.compaction.FileCompactor;
@@ -47,17 +47,17 @@
 
 public class MinorCompactor extends FileCompactor {
 
-  private static final SecureRandom random = new SecureRandom();
   private static final Logger log = LoggerFactory.getLogger(MinorCompactor.class);
 
   private final TabletServer tabletServer;
   private final MinorCompactionReason mincReason;
 
   public MinorCompactor(TabletServer tabletServer, Tablet tablet, InMemoryMap imm,
-      TabletFile outputFile, MinorCompactionReason mincReason, TableConfiguration tableConfig) {
+      ReferencedTabletFile outputFile, MinorCompactionReason mincReason,
+      TableConfiguration tableConfig) {
     super(tabletServer.getContext(), tablet.getExtent(), Collections.emptyMap(), outputFile, true,
         new MinCEnv(mincReason, imm.compactionIterator()), Collections.emptyList(), tableConfig,
-        tableConfig.getCryptoService());
+        tableConfig.getCryptoService(), tabletServer.getPausedCompactionMetrics());
     this.tabletServer = tabletServer;
     this.mincReason = mincReason;
   }
@@ -79,10 +79,10 @@
 
   @Override
   public CompactionStats call() {
-    final String outputFileName = getOutputFile();
+    final String outputFileName = getOutputFile().getMetadataPath();
     log.trace("Begin minor compaction {} {}", outputFileName, getExtent());
 
-    // output to new MapFile with a temporary name
+    // output to new data file with a temporary name
     int sleepTime = 100;
     double growthFactor = 4;
     int maxSleepTime = 1000 * 60 * 3; // 3 minutes
@@ -131,7 +131,7 @@
           throw new IllegalStateException(e);
         }
 
-        int sleep = sleepTime + random.nextInt(sleepTime);
+        int sleep = sleepTime + RANDOM.get().nextInt(sleepTime);
         log.debug("MinC failed sleeping {} ms before retrying", sleep);
         sleepUninterruptibly(sleep, TimeUnit.MILLISECONDS);
         sleepTime = (int) Math.round(Math.min(maxSleepTime, sleepTime * growthFactor));
@@ -146,7 +146,7 @@
         }
 
         if (isTableDeleting()) {
-          return new CompactionStats(0, 0);
+          return new CompactionStats(0, 0, 0);
         }
 
       } while (true);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index f17d9fa..730b55e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -41,7 +41,7 @@
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator.DataSource;
 import org.apache.accumulo.core.iteratorsImpl.system.StatsIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.util.Pair;
@@ -132,7 +132,7 @@
 
   private SortedKeyValueIterator<Key,Value> createIterator() throws IOException {
 
-    Map<TabletFile,DataFileValue> files;
+    Map<StoredTabletFile,DataFileValue> files;
 
     SamplerConfigurationImpl samplerConfig = scanParams.getSamplerConfigurationImpl();
 
@@ -168,20 +168,20 @@
       expectedDeletionCount = tablet.getDataSourceDeletions();
 
       memIters = tablet.getMemIterators(samplerConfig);
-      Pair<Long,Map<TabletFile,DataFileValue>> reservation = tablet.reserveFilesForScan();
+      Pair<Long,Map<StoredTabletFile,DataFileValue>> reservation = tablet.reserveFilesForScan();
       fileReservationId = reservation.getFirst();
       files = reservation.getSecond();
     }
 
-    Collection<InterruptibleIterator> mapfiles =
+    Collection<InterruptibleIterator> datafiles =
         fileManager.openFiles(files, scanParams.isIsolated(), samplerConfig);
 
-    List.of(mapfiles, memIters).forEach(c -> c.forEach(ii -> ii.setInterruptFlag(interruptFlag)));
+    List.of(datafiles, memIters).forEach(c -> c.forEach(ii -> ii.setInterruptFlag(interruptFlag)));
 
     List<SortedKeyValueIterator<Key,Value>> iters =
-        new ArrayList<>(mapfiles.size() + memIters.size());
+        new ArrayList<>(datafiles.size() + memIters.size());
 
-    iters.addAll(mapfiles);
+    iters.addAll(datafiles);
     iters.addAll(memIters);
 
     MultiIterator multiIter = new MultiIterator(iters, tablet.getExtent());
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
index 2b89a20..5e6d1a9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
@@ -157,7 +157,7 @@
     } finally {
       try {
         // code in finally block because always want
-        // to return mapfiles, even when exception is thrown
+        // to return datafiles, even when exception is thrown
         if (dataSource != null) {
           if (sawException || !scanParams.isIsolated()) {
             dataSource.close(sawException);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SnapshotTablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SnapshotTablet.java
index 1f0f754..57c7036 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SnapshotTablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SnapshotTablet.java
@@ -27,7 +27,6 @@
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
@@ -97,10 +96,9 @@
 
   }
 
-  @SuppressWarnings({"rawtypes", "unchecked"})
   @Override
-  public Pair<Long,Map<TabletFile,DataFileValue>> reserveFilesForScan() {
-    return new Pair(0L, getDatafiles());
+  public Pair<Long,Map<StoredTabletFile,DataFileValue>> reserveFilesForScan() {
+    return new Pair<>(0L, getDatafiles());
   }
 
   @Override
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 6a646c4..90af6d4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -18,9 +18,9 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.stream.Collectors.toList;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -63,16 +63,16 @@
 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.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FilePrefix;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator;
 import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
@@ -81,11 +81,11 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanDispatch;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -93,19 +93,18 @@
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.compaction.CompactionStats;
+import org.apache.accumulo.server.compaction.PausedCompactionMetrics;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
 import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
 import org.apache.accumulo.server.util.FileUtil;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.accumulo.tserver.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.MinorCompactionReason;
@@ -215,21 +214,22 @@
   private final Rate scannedRate = new Rate(0.95);
 
   private long lastMinorCompactionFinishTime = 0;
-  private long lastMapFileImportTime = 0;
+  private long lastDataFileImportTime = 0;
 
   private volatile long numEntries = 0;
   private volatile long numEntriesInMemory = 0;
 
   // Files that are currently in the process of bulk importing. Access to this is protected by the
   // tablet lock.
-  private final Set<TabletFile> bulkImporting = new HashSet<>();
+  private final Set<ReferencedTabletFile> bulkImporting = new HashSet<>();
 
   // Files that were successfully bulk imported. Using a concurrent map supports non-locking
   // operations on the key set which is useful for the periodic task that cleans up completed bulk
   // imports for all tablets. However the values of this map are ArrayList which do not support
   // concurrency. This is ok because all operations on the values are done while the tablet lock is
   // held.
-  private final ConcurrentHashMap<Long,List<TabletFile>> bulkImported = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<Long,List<ReferencedTabletFile>> bulkImported =
+      new ConcurrentHashMap<>();
 
   private final int logId;
 
@@ -253,15 +253,17 @@
     return dirUri;
   }
 
-  TabletFile getNextMapFilename(String prefix) throws IOException {
+  ReferencedTabletFile getNextDataFilename(FilePrefix prefix) throws IOException {
     String extension = FileOperations.getNewFileExtension(tableConfiguration);
-    return new TabletFile(new Path(chooseTabletDir() + "/" + prefix
+    return new ReferencedTabletFile(new Path(chooseTabletDir() + "/" + prefix.toPrefix()
         + context.getUniqueNameAllocator().getNextName() + "." + extension));
   }
 
-  TabletFile getNextMapFilenameForMajc(boolean propagateDeletes) throws IOException {
-    String tmpFileName = getNextMapFilename(!propagateDeletes ? "A" : "C").getMetaInsert() + "_tmp";
-    return new TabletFile(new Path(tmpFileName));
+  ReferencedTabletFile getNextDataFilenameForMajc(boolean propagateDeletes) throws IOException {
+    String tmpFileName = getNextDataFilename(
+        !propagateDeletes ? FilePrefix.MAJOR_COMPACTION_ALL_FILES : FilePrefix.MAJOR_COMPACTION)
+        .insert().getMetadataPath() + "_tmp";
+    return new ReferencedTabletFile(new Path(tmpFileName));
   }
 
   private void checkTabletDir(Path path) throws IOException {
@@ -299,17 +301,14 @@
     this.logId = tabletServer.createLogId();
 
     // translate any volume changes
-    @SuppressWarnings("deprecation")
-    boolean replicationEnabled = org.apache.accumulo.core.replication.ReplicationConfigurationUtil
-        .isEnabled(extent, this.tableConfiguration);
     TabletFiles tabletPaths =
         new TabletFiles(data.getDirectoryName(), data.getLogEntries(), data.getDataFiles());
     tabletPaths = VolumeUtil.updateTabletVolumes(tabletServer.getContext(), tabletServer.getLock(),
-        extent, tabletPaths, replicationEnabled);
+        extent, tabletPaths);
 
     this.dirName = data.getDirectoryName();
 
-    for (Entry<Long,List<TabletFile>> entry : data.getBulkImported().entrySet()) {
+    for (Entry<Long,List<ReferencedTabletFile>> entry : data.getBulkImported().entrySet()) {
       this.bulkImported.put(entry.getKey(), new ArrayList<>(entry.getValue()));
     }
 
@@ -329,8 +328,8 @@
       final CommitSession commitSession = getTabletMemory().getCommitSession();
       try {
         Set<String> absPaths = new HashSet<>();
-        for (TabletFile ref : datafiles.keySet()) {
-          absPaths.add(ref.getPathStr());
+        for (StoredTabletFile ref : datafiles.keySet()) {
+          absPaths.add(ref.getNormalizedPathStr());
         }
 
         tabletServer.recover(this.getTabletServer().getVolumeManager(), extent, logEntries,
@@ -352,25 +351,11 @@
         }
         commitSession.updateMaxCommittedTime(tabletTime.getTime());
 
-        @SuppressWarnings("deprecation")
-        boolean replicationEnabledForTable =
-            org.apache.accumulo.core.replication.ReplicationConfigurationUtil.isEnabled(extent,
-                tabletServer.getTableConfiguration(extent));
         if (entriesUsedOnTablet.get() == 0) {
           log.debug("No replayed mutations applied, removing unused entries for {}", extent);
           MetadataTableUtil.removeUnusedWALEntries(getTabletServer().getContext(), extent,
               logEntries, tabletServer.getLock());
           logEntries.clear();
-        } else if (replicationEnabledForTable) {
-          // record that logs may have data for this extent
-          @SuppressWarnings("deprecation")
-          Status status = org.apache.accumulo.server.replication.StatusUtil.openWithUnknownLength();
-          for (LogEntry logEntry : logEntries) {
-            log.debug("Writing updated status to metadata table for {} {}", logEntry.filename,
-                ProtobufUtil.toString(status));
-            ReplicationTableUtil.updateFiles(tabletServer.getContext(), extent, logEntry.filename,
-                status);
-          }
         }
 
       } catch (Exception t) {
@@ -384,8 +369,7 @@
       // make some closed references that represent the recovered logs
       currentLogs = new HashSet<>();
       for (LogEntry logEntry : logEntries) {
-        currentLogs.add(new DfsLogger(tabletServer.getContext(), tabletServer.getServerConfig(),
-            logEntry.filename, logEntry.getColumnQualifier().toString()));
+        currentLogs.add(DfsLogger.fromLogEntry(logEntry));
       }
 
       rebuildReferencedLogs();
@@ -461,13 +445,14 @@
       throw e;
     } finally {
       // code in finally block because always want
-      // to return mapfiles, even when exception is thrown
+      // to return data files, even when exception is thrown
       dataSource.close(sawException);
     }
   }
 
-  DataFileValue minorCompact(InMemoryMap memTable, TabletFile tmpDatafile, TabletFile newDatafile,
-      long queued, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
+  DataFileValue minorCompact(InMemoryMap memTable, ReferencedTabletFile tmpDatafile,
+      ReferencedTabletFile newDatafile, long queued, CommitSession commitSession, long flushId,
+      MinorCompactionReason mincReason) {
     boolean failed = false;
     long start = System.currentTimeMillis();
     timer.incrementStatusMinor();
@@ -497,7 +482,7 @@
         var storedFile = getDatafileManager().bringMinorCompactionOnline(tmpDatafile, newDatafile,
             new DataFileValue(stats.getFileSize(), stats.getEntriesWritten()), commitSession,
             flushId);
-        storedFile.ifPresent(stf -> compactable.filesAdded(true, List.of(stf)));
+        storedFile.ifPresent(stf -> compactable.filesAdded());
       } catch (Exception e) {
         TraceUtil.setException(span2, e, true);
         throw e;
@@ -888,7 +873,7 @@
   }
 
   /**
-   * Closes the mapfiles associated with a Tablet. If saveState is true, a minor compaction is
+   * Closes the data files associated with a Tablet. If saveState is true, a minor compaction is
    * performed.
    */
   @Override
@@ -1075,7 +1060,7 @@
 
     getTabletMemory().close();
 
-    // close map files
+    // close data files
     getTabletResources().close();
 
     if (completeClose) {
@@ -1118,8 +1103,8 @@
       }
 
       if (!tabletMeta.getLogs().isEmpty()) {
-        String msg = "Closed tablet " + extent + " has walog entries in " + MetadataTable.NAME + " "
-            + tabletMeta.getLogs();
+        String msg = "Closed tablet " + extent + " has walog entries in "
+            + AccumuloTable.METADATA.tableName() + " " + tabletMeta.getLogs();
         log.error(msg);
         throw new RuntimeException(msg);
       }
@@ -1363,8 +1348,8 @@
   private boolean isFindSplitsSuppressed() {
     if (supressFindSplits) {
       if (timeOfLastMinCWhenFindSplitsWasSupressed != lastMinorCompactionFinishTime
-          || timeOfLastImportWhenFindSplitsWasSupressed != lastMapFileImportTime) {
-        // a minor compaction or map file import has occurred... check again
+          || timeOfLastImportWhenFindSplitsWasSupressed != lastDataFileImportTime) {
+        // a minor compaction or data file import has occurred... check again
         supressFindSplits = false;
       } else {
         // nothing changed, do not split
@@ -1381,7 +1366,7 @@
   private void suppressFindSplits() {
     supressFindSplits = true;
     timeOfLastMinCWhenFindSplitsWasSupressed = lastMinorCompactionFinishTime;
-    timeOfLastImportWhenFindSplitsWasSupressed = lastMapFileImportTime;
+    timeOfLastImportWhenFindSplitsWasSupressed = lastDataFileImportTime;
   }
 
   private static int longestCommonLength(Text text, Text beforeMid) {
@@ -1395,7 +1380,7 @@
 
   // encapsulates results of computations needed to make determinations about splits
   private static class SplitComputations {
-    final Set<TabletFile> inputFiles;
+    final Set<StoredTabletFile> inputFiles;
 
     // cached result of calling FileUtil.findMidpoint
     final SortedMap<Double,Key> midPoint;
@@ -1403,7 +1388,7 @@
     // the last row seen in the files, only set for the default tablet
     final Text lastRowForDefaultTablet;
 
-    private SplitComputations(Set<TabletFile> inputFiles, SortedMap<Double,Key> midPoint,
+    private SplitComputations(Set<StoredTabletFile> inputFiles, SortedMap<Double,Key> midPoint,
         Text lastRowForDefaultTablet) {
       this.inputFiles = inputFiles;
       this.midPoint = midPoint;
@@ -1429,7 +1414,7 @@
       return Optional.empty();
     }
 
-    Set<TabletFile> files = getDatafileManager().getFiles();
+    Set<StoredTabletFile> files = getDatafileManager().getFiles();
     SplitComputations lastComputation = lastSplitComputation.get();
     if (lastComputation != null && lastComputation.inputFiles.equals(files)) {
       // the last computation is still relevant
@@ -1449,15 +1434,13 @@
     // Only want one thread doing this computation at time for a tablet.
     if (splitComputationLock.tryLock()) {
       try {
-        SortedMap<Double,Key> midpoint =
-            FileUtil.findMidPoint(context, tableConfiguration, chooseTabletDir(),
-                extent.prevEndRow(), extent.endRow(), FileUtil.toPathStrings(files), .25, true);
+        SortedMap<Double,Key> midpoint = FileUtil.findMidPoint(context, tableConfiguration,
+            chooseTabletDir(), extent.prevEndRow(), extent.endRow(), files, .25, true);
 
         Text lastRow = null;
 
         if (extent.endRow() == null) {
-          Key lastKey = (Key) FileUtil.findLastKey(context, tableConfiguration, files);
-          lastRow = lastKey.getRow();
+          lastRow = FileUtil.findLastRow(context, tableConfiguration, files);
         }
 
         newComputation = new SplitComputations(files, midpoint, lastRow);
@@ -1580,12 +1563,11 @@
       }
     } else {
       Text tsp = new Text(sp);
-      var fileStrings = FileUtil.toPathStrings(getDatafileManager().getFiles());
       // This ratio is calculated before that tablet is closed and outside of a lock, so new files
       // could arrive before the tablet is closed and locked. That is okay as the ratio is an
       // estimate.
       var ratio = FileUtil.estimatePercentageLTE(context, tableConfiguration, chooseTabletDir(),
-          extent.prevEndRow(), extent.endRow(), fileStrings, tsp);
+          extent.prevEndRow(), extent.endRow(), getDatafileManager().getFiles(), tsp);
       splitPoint = new SplitRowSpec(ratio, tsp);
     }
 
@@ -1600,12 +1582,12 @@
     }
 
     // obtain this info outside of synch block since it will involve opening
-    // the map files... it is ok if the set of map files changes, because
-    // this info is used for optimization... it is ok if map files are missing
+    // the data files... it is ok if the set of data files changes, because
+    // this info is used for optimization... it is ok if data files are missing
     // from the set... can still query and insert into the tablet while this
-    // map file operation is happening
-    Map<TabletFile,FileUtil.FileInfo> firstAndLastRows = FileUtil.tryToGetFirstAndLastRows(context,
-        tableConfiguration, getDatafileManager().getFiles());
+    // data file operation is happening
+    Map<StoredTabletFile,FileUtil.FileInfo> firstAndLastRows = FileUtil
+        .tryToGetFirstAndLastRows(context, tableConfiguration, getDatafileManager().getFiles());
 
     synchronized (this) {
       // java needs tuples ...
@@ -1732,14 +1714,14 @@
     return splitCreationTime;
   }
 
-  public void importMapFiles(long tid, Map<TabletFile,MapFileInfo> fileMap, boolean setTime)
-      throws IOException {
-    Map<TabletFile,DataFileValue> entries = new HashMap<>(fileMap.size());
+  public void importDataFiles(long tid, Map<ReferencedTabletFile,DataFileInfo> fileMap,
+      boolean setTime) throws IOException {
+    Map<ReferencedTabletFile,DataFileValue> entries = new HashMap<>(fileMap.size());
     List<String> files = new ArrayList<>();
 
-    for (Entry<TabletFile,MapFileInfo> entry : fileMap.entrySet()) {
+    for (Entry<ReferencedTabletFile,DataFileInfo> entry : fileMap.entrySet()) {
       entries.put(entry.getKey(), new DataFileValue(entry.getValue().estimatedSize, 0L));
-      files.add(entry.getKey().getPathStr());
+      files.add(entry.getKey().getNormalizedPathStr());
     }
 
     // Clients timeout and will think that this operation failed.
@@ -1781,9 +1763,9 @@
             + " seconds to get tablet lock for " + extent + " " + tid);
       }
 
-      List<TabletFile> alreadyImported = bulkImported.get(tid);
+      List<ReferencedTabletFile> alreadyImported = bulkImported.get(tid);
       if (alreadyImported != null) {
-        for (TabletFile entry : alreadyImported) {
+        for (ReferencedTabletFile entry : alreadyImported) {
           if (fileMap.remove(entry) != null) {
             log.trace("Ignoring import of bulk file already imported: {}", entry);
           }
@@ -1802,8 +1784,8 @@
     try {
       tabletServer.updateBulkImportState(files, BulkImportState.LOADING);
 
-      var storedTabletFile = getDatafileManager().importMapFiles(tid, entries, setTime);
-      lastMapFileImportTime = System.currentTimeMillis();
+      getDatafileManager().importDataFiles(tid, entries, setTime);
+      lastDataFileImportTime = System.currentTimeMillis();
 
       synchronized (this) {
         // only mark the bulk import a success if no exception was thrown
@@ -1813,7 +1795,7 @@
       if (isSplitPossible()) {
         getTabletServer().executeSplit(this);
       } else {
-        compactable.filesAdded(false, storedTabletFile);
+        compactable.filesAdded();
       }
     } finally {
       synchronized (this) {
@@ -1872,10 +1854,7 @@
   public void checkIfMinorCompactionNeededForLogs(List<DfsLogger> closedLogs) {
 
     // grab this outside of tablet lock.
-    @SuppressWarnings("deprecation")
-    Property prop = tableConfiguration.resolve(Property.TSERV_WAL_MAX_REFERENCED,
-        Property.TSERV_WALOG_MAX_REFERENCED, Property.TABLE_MINC_LOGS_MAX);
-    int maxLogs = tableConfiguration.getCount(prop);
+    int maxLogs = tableConfiguration.getCount(Property.TSERV_WAL_MAX_REFERENCED);
 
     String reason = null;
     synchronized (this) {
@@ -1890,7 +1869,7 @@
         List<DfsLogger> oldClosed = closedLogs.subList(0, closedLogs.size() - maxLogs);
         for (DfsLogger closedLog : oldClosed) {
           if (currentLogs.contains(closedLog)) {
-            reason = "referenced at least one old write ahead log " + closedLog.getFileName();
+            reason = "referenced at least one old write ahead log " + closedLog.getLogEntry();
             break;
           }
         }
@@ -1908,12 +1887,12 @@
     return logLock;
   }
 
-  Set<String> beginClearingUnusedLogs() {
+  Set<LogEntry> beginClearingUnusedLogs() {
     Preconditions.checkState(logLock.isHeldByCurrentThread());
-    Set<String> unusedLogs = new HashSet<>();
+    Set<LogEntry> unusedLogs = new HashSet<>();
 
-    ArrayList<String> otherLogsCopy = new ArrayList<>();
-    ArrayList<String> currentLogsCopy = new ArrayList<>();
+    ArrayList<LogEntry> otherLogsCopy = new ArrayList<>();
+    ArrayList<LogEntry> currentLogsCopy = new ArrayList<>();
 
     synchronized (this) {
       if (removingLogs) {
@@ -1922,13 +1901,13 @@
       }
 
       for (DfsLogger logger : otherLogs) {
-        otherLogsCopy.add(logger.toString());
-        unusedLogs.add(logger.getMeta());
+        otherLogsCopy.add(logger.getLogEntry());
+        unusedLogs.add(logger.getLogEntry());
       }
 
       for (DfsLogger logger : currentLogs) {
-        currentLogsCopy.add(logger.toString());
-        unusedLogs.remove(logger.getMeta());
+        currentLogsCopy.add(logger.getLogEntry());
+        unusedLogs.remove(logger.getLogEntry());
       }
 
       if (!unusedLogs.isEmpty()) {
@@ -1937,16 +1916,16 @@
     }
 
     // do debug logging outside tablet lock
-    for (String logger : otherLogsCopy) {
-      log.trace("Logs for memory compacted: {} {}", getExtent(), logger);
+    for (LogEntry logEntry : otherLogsCopy) {
+      log.trace("Logs for memory compacted: {} {}", getExtent(), logEntry);
     }
 
-    for (String logger : currentLogsCopy) {
-      log.trace("Logs for current memory: {} {}", getExtent(), logger);
+    for (LogEntry logEntry : currentLogsCopy) {
+      log.trace("Logs for current memory: {} {}", getExtent(), logEntry);
     }
 
-    for (String logger : unusedLogs) {
-      log.trace("Logs to be destroyed: {} {}", getExtent(), logger);
+    for (LogEntry logEntry : unusedLogs) {
+      log.trace("Logs to be destroyed: {} {}", getExtent(), logEntry);
     }
 
     return unusedLogs;
@@ -2040,10 +2019,6 @@
     logLock.unlock();
   }
 
-  public void chopFiles() {
-    compactable.initiateChop();
-  }
-
   public void compactAll(long compactionId, CompactionConfig compactionConfig) {
 
     synchronized (this) {
@@ -2099,7 +2074,7 @@
   }
 
   public Map<StoredTabletFile,DataFileValue> updatePersistedTime(long bulkTime,
-      Map<TabletFile,DataFileValue> paths, long tid) {
+      Map<ReferencedTabletFile,DataFileValue> paths, long tid) {
     synchronized (timeLock) {
       if (bulkTime > persistedTime) {
         persistedTime = bulkTime;
@@ -2116,7 +2091,8 @@
    * Update tablet file data from flush. Returns a StoredTabletFile if there are data entries.
    */
   public Optional<StoredTabletFile> updateTabletDataFile(long maxCommittedTime,
-      TabletFile newDatafile, DataFileValue dfv, Set<String> unusedWalLogs, long flushId) {
+      ReferencedTabletFile newDatafile, DataFileValue dfv, Set<LogEntry> unusedWalLogs,
+      long flushId) {
     synchronized (timeLock) {
       if (maxCommittedTime > persistedTime) {
         persistedTime = maxCommittedTime;
@@ -2140,12 +2116,16 @@
     return getTabletServer().getScanMetrics();
   }
 
+  public PausedCompactionMetrics getPausedCompactionMetrics() {
+    return getTabletServer().getPausedCompactionMetrics();
+  }
+
   DatafileManager getDatafileManager() {
     return datafileManager;
   }
 
   @Override
-  public Pair<Long,Map<TabletFile,DataFileValue>> reserveFilesForScan() {
+  public Pair<Long,Map<StoredTabletFile,DataFileValue>> reserveFilesForScan() {
     return getDatafileManager().reserveFilesForScan();
   }
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java
index 1289010..dd23c5a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java
@@ -44,7 +44,6 @@
 import org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.ColumnVisibility;
@@ -54,6 +53,7 @@
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.TooManyFilesException;
+import org.apache.accumulo.server.mem.LowMemoryDetector.DetectionScope;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.TabletHostingServer;
 import org.apache.accumulo.tserver.TabletServerResourceManager;
@@ -87,10 +87,13 @@
 
   protected final TableConfiguration tableConfiguration;
 
+  private final boolean isUserTable;
+
   public TabletBase(TabletHostingServer server, KeyExtent extent) {
     this.context = server.getContext();
     this.server = server;
     this.extent = extent;
+    this.isUserTable = !extent.isMeta();
 
     TableConfiguration tblConf = context.getTableConfiguration(extent.tableId());
     if (tblConf == null) {
@@ -125,7 +128,7 @@
 
   public abstract void returnMemIterators(List<InMemoryMap.MemoryIterator> iters);
 
-  public abstract Pair<Long,Map<TabletFile,DataFileValue>> reserveFilesForScan();
+  public abstract Pair<Long,Map<StoredTabletFile,DataFileValue>> reserveFilesForScan();
 
   public abstract void returnFilesForScan(long scanId);
 
@@ -213,7 +216,7 @@
       throw e;
     } finally {
       // code in finally block because always want
-      // to return mapfiles, even when exception is thrown
+      // to return data files, even when exception is thrown
       dataSource.close(sawException);
 
       synchronized (this) {
@@ -230,7 +233,19 @@
   Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, ScanParameters scanParams)
       throws IOException {
 
-    // log.info("In nextBatch..");
+    while (context.getLowMemoryDetector().isRunningLowOnMemory(context, DetectionScope.SCAN, () -> {
+      return isUserTable;
+    }, () -> {
+      log.info("Not starting next batch because low on memory, extent: {}", extent);
+      server.getScanMetrics().incrementScanPausedForLowMemory();
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(
+            "Interrupted while waiting for low memory condition to resolve", e);
+      }
+    })) {}
 
     long batchTimeOut = scanParams.getBatchTimeOut();
 
@@ -280,7 +295,15 @@
 
       boolean timesUp = batchTimeOut > 0 && (System.nanoTime() - startNanos) >= timeToRun;
 
-      if (resultSize >= maxResultsSize || results.size() >= scanParams.getMaxEntries() || timesUp) {
+      boolean runningLowOnMemory =
+          context.getLowMemoryDetector().isRunningLowOnMemory(context, DetectionScope.SCAN, () -> {
+            return isUserTable;
+          }, () -> {
+            log.info("Not continuing next batch because low on memory, extent: {}", extent);
+            server.getScanMetrics().incrementEarlyReturnForLowMemory();
+          });
+      if (runningLowOnMemory || resultSize >= maxResultsSize
+          || results.size() >= scanParams.getMaxEntries() || timesUp) {
         continueKey = new Key(key);
         skipContinueKey = true;
         break;
@@ -319,6 +342,20 @@
   private Tablet.LookupResult lookup(SortedKeyValueIterator<Key,Value> mmfi, List<Range> ranges,
       List<KVEntry> results, ScanParameters scanParams, long maxResultsSize) throws IOException {
 
+    while (context.getLowMemoryDetector().isRunningLowOnMemory(context, DetectionScope.SCAN, () -> {
+      return isUserTable;
+    }, () -> {
+      log.info("Not starting lookup because low on memory, extent: {}", extent);
+      server.getScanMetrics().incrementScanPausedForLowMemory();
+      try {
+        Thread.sleep(500);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException(
+            "Interrupted while waiting for low memory condition to resolve", e);
+      }
+    })) {}
+
     Tablet.LookupResult lookupResult = new Tablet.LookupResult();
 
     boolean exceededMemoryUsage = false;
@@ -347,7 +384,14 @@
 
       boolean timesUp = batchTimeOut > 0 && (System.nanoTime() - startNanos) > timeToRun;
 
-      if (exceededMemoryUsage || tabletClosed || timesUp || yielded) {
+      boolean runningLowOnMemory =
+          context.getLowMemoryDetector().isRunningLowOnMemory(context, DetectionScope.SCAN, () -> {
+            return isUserTable;
+          }, () -> {
+            log.info("Not continuing lookup because low on memory, extent: {}", extent);
+            server.getScanMetrics().incrementEarlyReturnForLowMemory();
+          });
+      if (runningLowOnMemory || exceededMemoryUsage || tabletClosed || timesUp || yielded) {
         lookupResult.unfinishedRanges.add(range);
         continue;
       }
@@ -378,7 +422,14 @@
 
           timesUp = batchTimeOut > 0 && (System.nanoTime() - startNanos) > timeToRun;
 
-          if (exceededMemoryUsage || timesUp) {
+          runningLowOnMemory = context.getLowMemoryDetector().isRunningLowOnMemory(context,
+              DetectionScope.SCAN, () -> {
+                return isUserTable;
+              }, () -> {
+                log.info("Not continuing lookup because low on memory, extent: {}", extent);
+                server.getScanMetrics().incrementEarlyReturnForLowMemory();
+              });
+          if (runningLowOnMemory || exceededMemoryUsage || timesUp) {
             addUnfinishedRange(lookupResult, range, key);
             break;
           }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
index 1614715..5963dc1 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletData.java
@@ -26,8 +26,8 @@
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
@@ -47,7 +47,7 @@
   private long flushID = -1;
   private long compactID = -1;
   private Location lastLocation = null;
-  private Map<Long,List<TabletFile>> bulkImported = new HashMap<>();
+  private Map<Long,List<ReferencedTabletFile>> bulkImported = new HashMap<>();
   private long splitTime = 0;
   private String directoryName = null;
   private Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions;
@@ -69,7 +69,7 @@
     dataFiles.putAll(meta.getFilesMap());
 
     meta.getLoaded().forEach((path, txid) -> {
-      bulkImported.computeIfAbsent(txid, k -> new ArrayList<>()).add(path);
+      bulkImported.computeIfAbsent(txid, k -> new ArrayList<>()).add(path.getTabletFile());
     });
 
     this.extCompactions = meta.getExternalCompactions();
@@ -78,7 +78,7 @@
   // Data pulled from an existing tablet to make a split
   public TabletData(String dirName, SortedMap<StoredTabletFile,DataFileValue> highDatafileSizes,
       MetadataTime time, long lastFlushID, long lastCompactID, Location lastLocation,
-      Map<Long,List<TabletFile>> bulkIngestedFiles) {
+      Map<Long,List<ReferencedTabletFile>> bulkIngestedFiles) {
     this.directoryName = dirName;
     this.dataFiles = highDatafileSizes;
     this.time = time;
@@ -118,7 +118,7 @@
     return lastLocation;
   }
 
-  public Map<Long,List<TabletFile>> getBulkImported() {
+  public Map<Long,List<ReferencedTabletFile>> getBulkImported() {
     return bulkImported;
   }
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/util/CreateEmpty.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/util/CreateEmpty.java
new file mode 100644
index 0000000..2f15d2a
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/util/CreateEmpty.java
@@ -0,0 +1,198 @@
+/*
+ * 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.tserver.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.tserver.log.DfsLogger.LOG_FILE_HEADER_V4;
+import static org.apache.accumulo.tserver.logger.LogEvents.OPEN;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.cli.ConfigOpts;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.crypto.CryptoEnvironmentImpl;
+import org.apache.accumulo.core.crypto.CryptoUtils;
+import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.file.rfile.RFileOperations;
+import org.apache.accumulo.core.file.rfile.bcfile.Compression;
+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.spi.file.rfile.compression.NoCompression;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.start.spi.KeywordExecutable;
+import org.apache.accumulo.tserver.logger.LogFileKey;
+import org.apache.accumulo.tserver.logger.LogFileValue;
+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 final String RF_EXTENSION = ".rf";
+  public static final String WAL_EXTENSION = ".wal";
+
+  public static class MatchesValidFileExtension implements IParameterValidator {
+    @Override
+    public void validate(String name, String value) throws ParameterException {
+      if (value.endsWith(RF_EXTENSION) || value.endsWith(WAL_EXTENSION)) {
+        return;
+      }
+      throw new ParameterException("File must end with either " + RF_EXTENSION + " or "
+          + WAL_EXTENSION + " 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 ConfigOpts {
+    @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 = MatchesValidFileExtension.class)
+    List<String> files = new ArrayList<>();
+
+    public enum OutFileType {
+      RF, WAL
+    }
+
+    // rfile as default keeps previous behaviour
+    @Parameter(names = "--type")
+    public OutFileType fileType = OutFileType.RF;
+
+  }
+
+  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 empty RFiles (RF) or empty write-ahead log (WAL) files for emergency recovery";
+  }
+
+  @Override
+  public void execute(String[] args) throws Exception {
+
+    Opts opts = new Opts();
+    opts.parseArgs("accumulo create-empty", args);
+
+    var siteConfig = opts.getSiteConfiguration();
+    try (ServerContext context = new ServerContext(siteConfig)) {
+      switch (opts.fileType) {
+        case RF:
+          createEmptyRFile(opts, context);
+          break;
+        case WAL:
+          createEmptyWal(opts, context);
+          break;
+        default:
+          throw new ParameterException("file type must be RF or WAL, received: " + opts.fileType);
+      }
+    }
+  }
+
+  void createEmptyRFile(final Opts opts, final ServerContext context) throws IOException {
+    var vm = context.getVolumeManager();
+
+    CryptoEnvironment env = new CryptoEnvironmentImpl(CryptoEnvironment.Scope.TABLE);
+    CryptoService cryptoService = context.getCryptoFactory().getService(env,
+        context.getConfiguration().getAllCryptoProperties());
+
+    for (String filename : opts.files) {
+      Path path = new Path(filename);
+      checkFileExists(path, vm);
+      UnreferencedTabletFile tabletFile =
+          UnreferencedTabletFile.of(vm.getFileSystemByPath(path), path);
+      LOG.info("Writing to file '{}'", tabletFile);
+      FileSKVWriter writer = new RFileOperations().newWriterBuilder()
+          .forFile(tabletFile, vm.getFileSystemByPath(path), context.getHadoopConf(), cryptoService)
+          .withTableConfiguration(DefaultConfiguration.getInstance()).withCompression(opts.codec)
+          .build();
+      writer.close();
+    }
+  }
+
+  void createEmptyWal(Opts opts, ServerContext context) throws IOException {
+    final LogFileValue EMPTY = new LogFileValue();
+
+    var vm = context.getVolumeManager();
+
+    for (String filename : opts.files) {
+      Path path = new Path(filename);
+      checkFileExists(path, vm);
+      try (var out = new DataOutputStream(vm.create(path))) {
+        LOG.info("Output file: {}", path);
+
+        out.write(LOG_FILE_HEADER_V4.getBytes(UTF_8));
+
+        CryptoEnvironment env = new CryptoEnvironmentImpl(CryptoEnvironment.Scope.WAL);
+        CryptoService cryptoService = context.getCryptoFactory().getService(env,
+            context.getConfiguration().getAllCryptoProperties());
+
+        byte[] cryptoParams = cryptoService.getFileEncrypter(env).getDecryptionParameters();
+        CryptoUtils.writeParams(cryptoParams, out);
+
+        LogFileKey key = new LogFileKey();
+        key.event = OPEN;
+        key.tserverSession = "";
+
+        key.write(out);
+        EMPTY.write(out);
+      }
+    }
+  }
+
+  private void checkFileExists(final Path path, final VolumeManager vm) throws IOException {
+    if (vm.exists(path)) {
+      throw new IllegalArgumentException(path + " exists");
+    }
+  }
+}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
index 6304377..cad9d15 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
@@ -33,7 +33,9 @@
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
 import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
@@ -44,11 +46,9 @@
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TRange;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.tserver.ScanServer.ScanReservation;
 import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
 import org.apache.accumulo.tserver.tablet.SnapshotTablet;
@@ -57,6 +57,8 @@
 import org.apache.thrift.TException;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class ScanServerTest {
 
   public class TestScanServer extends ScanServer {
@@ -65,7 +67,7 @@
     private TabletResolver resolver;
     private ScanReservation reservation;
 
-    protected TestScanServer(ScanServerOpts opts, String[] args) {
+    protected TestScanServer(ConfigOpts opts, String[] args) {
       super(opts, args);
     }
 
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
index 7bb8cf2..2e682b4 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/WalRemovalOrderTest.java
@@ -25,33 +25,20 @@
 import java.util.List;
 import java.util.Set;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.tserver.log.DfsLogger;
-import org.apache.accumulo.tserver.log.DfsLogger.ServerResources;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Sets;
 
 public class WalRemovalOrderTest {
 
-  private static DfsLogger mockLogger(String filename) {
-    ServerResources conf = new ServerResources() {
-      @Override
-      public AccumuloConfiguration getConfiguration() {
-        return DefaultConfiguration.getInstance();
-      }
-
-      @Override
-      public VolumeManager getVolumeManager() {
-        throw new UnsupportedOperationException();
-      }
-    };
-    return new DfsLogger(null, conf, filename, null);
+  private DfsLogger mockLogger(String filename) {
+    var mockLogEntry = LogEntry.fromPath(filename + "+1234/11111111-1111-1111-1111-111111111111");
+    return DfsLogger.fromLogEntry(mockLogEntry);
   }
 
-  private static LinkedHashSet<DfsLogger> mockLoggers(String... logs) {
+  private LinkedHashSet<DfsLogger> mockLoggers(String... logs) {
     LinkedHashSet<DfsLogger> logSet = new LinkedHashSet<>();
 
     for (String log : logs) {
@@ -61,7 +48,7 @@
     return logSet;
   }
 
-  private static void runTest(LinkedHashSet<DfsLogger> closedLogs, Set<DfsLogger> inUseLogs,
+  private void runTest(LinkedHashSet<DfsLogger> closedLogs, Set<DfsLogger> inUseLogs,
       Set<DfsLogger> expected) {
     Set<DfsLogger> eligible = TabletServer.findOldestUnreferencedWals(List.copyOf(closedLogs),
         candidates -> candidates.removeAll(inUseLogs));
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactableUtilsTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactableUtilsTest.java
index 3ec01b0..84229c7 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactableUtilsTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactableUtilsTest.java
@@ -20,7 +20,7 @@
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.tserver.tablet.CompactableUtils;
 import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.Test;
@@ -29,10 +29,11 @@
 
   @Test
   public void testEquivalence() {
-    TabletFile expected = new TabletFile(
+    ReferencedTabletFile expected = new ReferencedTabletFile(
         new Path("hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf"));
-    TabletFile tmpFile = new TabletFile(new Path(expected.getMetaInsert() + "_tmp"));
-    TabletFile dest = CompactableUtils.computeCompactionFileDest(tmpFile);
+    ReferencedTabletFile tmpFile =
+        new ReferencedTabletFile(new Path(expected.getNormalizedPathStr() + "_tmp"));
+    ReferencedTabletFile dest = CompactableUtils.computeCompactionFileDest(tmpFile);
     assertEquals(expected, dest);
   }
 
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactionPlanTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactionPlanTest.java
deleted file mode 100644
index d88d4ec..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/CompactionPlanTest.java
+++ /dev/null
@@ -1,82 +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.tserver.compaction;
-
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import java.util.Set;
-
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("removal")
-public class CompactionPlanTest {
-
-  @Test
-  public void testOverlappingInputAndDelete() {
-    CompactionPlan cp1 = new CompactionPlan();
-
-    StoredTabletFile fr1 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-1/1.rf");
-    StoredTabletFile fr2 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-1/2.rf");
-
-    cp1.inputFiles.add(fr1);
-
-    cp1.deleteFiles.add(fr1);
-    cp1.deleteFiles.add(fr2);
-
-    Set<StoredTabletFile> allFiles = Set.of(fr1, fr2);
-
-    assertThrows(IllegalStateException.class, () -> cp1.validate(allFiles));
-  }
-
-  @Test
-  public void testInputNotInAllFiles() {
-    CompactionPlan cp1 = new CompactionPlan();
-
-    StoredTabletFile fr1 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-1/1.rf");
-    StoredTabletFile fr2 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-1/2.rf");
-    StoredTabletFile fr3 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-2/3.rf");
-
-    cp1.inputFiles.add(fr1);
-    cp1.inputFiles.add(fr2);
-    cp1.inputFiles.add(fr3);
-
-    Set<StoredTabletFile> allFiles = Set.of(fr1, fr2);
-
-    assertThrows(IllegalStateException.class, () -> cp1.validate(allFiles));
-  }
-
-  @Test
-  public void testDeleteNotInAllFiles() {
-    CompactionPlan cp1 = new CompactionPlan();
-
-    StoredTabletFile fr1 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-1/1.rf");
-    StoredTabletFile fr2 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-1/2.rf");
-    StoredTabletFile fr3 = new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-2/3.rf");
-
-    cp1.deleteFiles.add(fr1);
-    cp1.deleteFiles.add(fr2);
-    cp1.deleteFiles.add(fr3);
-
-    Set<StoredTabletFile> allFiles = Set.of(fr1, fr2);
-
-    assertThrows(IllegalStateException.class, () -> cp1.validate(allFiles));
-  }
-
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
deleted file mode 100644
index 90d2d78..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/DefaultCompactionStrategyTest.java
+++ /dev/null
@@ -1,347 +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.tserver.compaction;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.LongSummaryStatistics;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-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.data.Key;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("removal")
-public class DefaultCompactionStrategyTest {
-
-  private static Pair<Key,Key> keys(String firstString, String secondString) {
-    Key first = null;
-    if (firstString != null) {
-      first = new Key(new Text(firstString));
-    }
-    Key second = null;
-    if (secondString != null) {
-      second = new Key(new Text(secondString));
-    }
-    return new Pair<>(first, second);
-  }
-
-  public static ServerContext getServerContext() {
-    ServerContext context = EasyMock.createMock(ServerContext.class);
-    EasyMock.replay(context);
-    return context;
-  }
-
-  static final Map<String,Pair<Key,Key>> fakeFiles = new HashMap<>();
-
-  static {
-    fakeFiles.put("file1", keys("b", "m"));
-    fakeFiles.put("file2", keys("n", "z"));
-    fakeFiles.put("file3", keys("a", "y"));
-    fakeFiles.put("file4", keys(null, null));
-  }
-
-  static final DefaultConfiguration dfault = DefaultConfiguration.getInstance();
-
-  private static class TestCompactionRequest extends MajorCompactionRequest {
-
-    Integer mfpt = null;
-
-    TestCompactionRequest(KeyExtent extent, MajorCompactionReason reason,
-        Map<StoredTabletFile,DataFileValue> files) {
-      super(extent, reason, dfault, getServerContext());
-      setFiles(files);
-    }
-
-    TestCompactionRequest(KeyExtent extent, MajorCompactionReason reason,
-        Map<StoredTabletFile,DataFileValue> files, AccumuloConfiguration config) {
-      super(extent, reason, config, getServerContext());
-      setFiles(files);
-    }
-
-    public void setMaxFilesPerTablet(int mfpt) {
-      this.mfpt = mfpt;
-    }
-
-    @Override
-    public int getMaxFilesPerTablet() {
-      if (mfpt != null) {
-        return mfpt;
-      }
-      return super.getMaxFilesPerTablet();
-    }
-
-  }
-
-  static Map<StoredTabletFile,DataFileValue> createFileMap(Object... objs) {
-    Map<StoredTabletFile,DataFileValue> files = new HashMap<>();
-    for (int i = 0; i < objs.length; i += 2) {
-      files.put(new StoredTabletFile("hdfs://nn1/accumulo/tables/5/t-0001/" + objs[i]),
-          new DataFileValue(((Number) objs[i + 1]).longValue(), 0));
-    }
-    return files;
-  }
-
-  private TestCompactionRequest createRequest(MajorCompactionReason reason, Object... objs) {
-    return createRequest(new KeyExtent(TableId.of("0"), null, null), reason, objs);
-  }
-
-  private TestCompactionRequest createRequest(KeyExtent extent, MajorCompactionReason reason,
-      Object... objs) {
-    return new TestCompactionRequest(extent, reason, createFileMap(objs));
-  }
-
-  private static Set<String> asSet(String... strings) {
-    return asSet(Arrays.asList(strings));
-  }
-
-  private static Set<String> asStringSet(Collection<StoredTabletFile> refs) {
-    HashSet<String> result = new HashSet<>();
-    for (TabletFile ref : refs) {
-      result.add(ref.getPathStr());
-    }
-    return result;
-  }
-
-  private static Set<String> asSet(Collection<String> strings) {
-    HashSet<String> result = new HashSet<>();
-    for (String string : strings) {
-      result.add("hdfs://nn1/accumulo/tables/5/t-0001/" + string);
-    }
-    return result;
-  }
-
-  @Test
-  public void testGetCompactionPlan() throws Exception {
-
-    // test are expecting this default
-    assertEquals(10,
-        DefaultConfiguration.getInstance().getCount(Property.TSERV_MAJC_THREAD_MAXOPEN));
-
-    DefaultCompactionStrategy s = new DefaultCompactionStrategy();
-
-    // do nothing
-    TestCompactionRequest request =
-        createRequest(MajorCompactionReason.IDLE, "file1", 10, "file2", 10);
-    s.gatherInformation(request);
-    CompactionPlan plan = s.getCompactionPlan(request);
-    assertTrue(plan.inputFiles.isEmpty());
-
-    // do everything
-    request = createRequest(MajorCompactionReason.IDLE, "file1", 10, "file2", 10, "file3", 10);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(3, plan.inputFiles.size());
-
-    // do everything
-    request = createRequest(MajorCompactionReason.USER, "file1", 10, "file2", 10);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(2, plan.inputFiles.size());
-
-    // partial
-    request = createRequest(MajorCompactionReason.NORMAL, "file0", 100, "file1", 10, "file2", 10,
-        "file3", 10);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(3, plan.inputFiles.size());
-    assertEquals(asStringSet(plan.inputFiles), asSet("file1,file2,file3".split(",")));
-
-    // Two windows (of size 10 or less) meet the compaction criteria. Should select the smallest set
-    // of files that meet the criteria.
-    request = createRequest(MajorCompactionReason.NORMAL, "file0", 100, "file1", 100, "file2", 100,
-        "file3", 10, "file4", 10, "file5", 10, "file6", 10, "file7", 10, "file8", 10, "file9", 10,
-        "fileA", 10);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(8, plan.inputFiles.size());
-    assertEquals(asStringSet(plan.inputFiles),
-        asSet("file3,file4,file5,file6,file7,file8,file9,fileA".split(",")));
-
-    // The last 10 files do not meet compaction ratio critea. Should move window of 10 files up
-    // looking for files that meet criteria.
-    request = createRequest(MajorCompactionReason.NORMAL, "file0", 19683, "file1", 19683, "file2",
-        19683, "file3", 6561, "file4", 2187, "file5", 729, "file6", 243, "file7", 81, "file8", 27,
-        "file9", 9, "fileA", 3, "fileB", 1);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(10, plan.inputFiles.size());
-    assertEquals(asStringSet(plan.inputFiles),
-        asSet("file0,file1,file2,file3,file4,file5,file6,file7,file8,file9".split(",")));
-
-    // No window of files meets the compaction criteria, but there are more files than the max
-    // allowed. Should compact the smallest 2.
-    request = createRequest(MajorCompactionReason.NORMAL, "file1", 19683, "file2", 19683, "file3",
-        6561, "file4", 2187, "file5", 729, "file6", 243, "file7", 81, "file8", 27, "file9", 9,
-        "fileA", 3, "fileB", 1);
-    request.setMaxFilesPerTablet(10);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(2, plan.inputFiles.size());
-    assertEquals(asStringSet(plan.inputFiles), asSet("fileA,fileB".split(",")));
-
-    // The last 9 files meet the compaction criteria, but 10 files need to be compacted. Should move
-    // window of 10 files up looking for files that meet criteria.
-    request = createRequest(MajorCompactionReason.NORMAL, "file01", 1500, "file02", 1400, "file03",
-        1300, "file04", 1200, "file05", 1100, "file06", 1000, "file07", 900, "file08", 800,
-        "file09", 700, "file10", 600, "file11", 500, "file12", 400, "file13", 400, "file14", 300,
-        "file15", 200, "file16", 100, "file17", 9, "file18", 8, "file19", 7, "file20", 6, "file21",
-        5, "file22", 4, "file23", 3, "file24", 2, "file25", 1);
-    request.setMaxFilesPerTablet(15);
-    s.gatherInformation(request);
-    plan = s.getCompactionPlan(request);
-    assertEquals(10, plan.inputFiles.size());
-    assertEquals(asStringSet(plan.inputFiles),
-        asSet("file12,file13,file14,file15,file16,file17,file18,file19,file20,file21".split(",")));
-
-  }
-
-  class SimulatedTablet {
-
-    private int maxFilesPerTablet;
-    private ConfigurationCopy config;
-
-    int nextFile = 0;
-    Map<StoredTabletFile,DataFileValue> files = new HashMap<>();
-
-    long totalRead = 0;
-    long added = 0;
-
-    SimulatedTablet(int maxFilesToCompact, int maxFilesPertablet) {
-      this.maxFilesPerTablet = maxFilesPertablet;
-
-      config = new ConfigurationCopy(DefaultConfiguration.getInstance());
-      config.set(Property.TSERV_MAJC_THREAD_MAXOPEN, maxFilesToCompact + "");
-    }
-
-    void addFiles(int num, int size, int entries) {
-      for (int i = 0; i < num; i++) {
-        String name =
-            "hdfs://nn1/accumulo/tables/5/t-0001/I" + String.format("%06d", nextFile) + ".rf";
-        nextFile++;
-
-        files.put(new StoredTabletFile(name), new DataFileValue(size, entries));
-        added += size;
-      }
-    }
-
-    long compact(MajorCompactionReason reason) {
-      TestCompactionRequest request = new TestCompactionRequest(
-          new KeyExtent(TableId.of("0"), (Text) null, null), reason, files, config);
-      request.setMaxFilesPerTablet(maxFilesPerTablet);
-
-      DefaultCompactionStrategy s = new DefaultCompactionStrategy();
-
-      if (s.shouldCompact(request)) {
-        CompactionPlan plan = s.getCompactionPlan(request);
-
-        long totalSize = 0;
-        long totalEntries = 0;
-
-        for (StoredTabletFile fr : plan.inputFiles) {
-          DataFileValue dfv = files.remove(fr);
-
-          totalSize += dfv.getSize();
-          totalEntries += dfv.getNumEntries();
-
-          totalRead += dfv.getSize();
-        }
-
-        String name =
-            "hdfs://nn1/accumulo/tables/5/t-0001/C" + String.format("%06d", nextFile) + ".rf";
-        nextFile++;
-
-        files.put(new StoredTabletFile(name), new DataFileValue(totalSize, totalEntries));
-
-        return totalSize;
-
-      } else {
-        return 0;
-      }
-    }
-
-    long getTotalRead() {
-      return totalRead;
-    }
-
-    public long getTotalAdded() {
-      return added;
-    }
-
-    void print() {
-      List<Entry<StoredTabletFile,DataFileValue>> entries = new ArrayList<>(files.entrySet());
-
-      entries.sort((e1, e2) -> Long.compare(e2.getValue().getSize(), e1.getValue().getSize()));
-
-      for (Entry<StoredTabletFile,DataFileValue> entry : entries) {
-
-        System.out.printf("%s %,d %,d\n", entry.getKey().getFileName(), entry.getValue().getSize(),
-            entry.getValue().getNumEntries());
-      }
-    }
-
-    public int getNumFiles() {
-      return files.size();
-    }
-  }
-
-  @Test
-  public void simulationTest() throws Exception {
-    for (int n = 1; n < 10; n++) {
-      LongSummaryStatistics lss = new LongSummaryStatistics();
-      SimulatedTablet simuTablet = new SimulatedTablet(10, 15);
-
-      for (int i = 0; i < 1000; i++) {
-        simuTablet.addFiles(n, 1000, 10);
-
-        simuTablet.compact(MajorCompactionReason.NORMAL);
-
-        lss.accept(simuTablet.getNumFiles());
-      }
-
-      while (simuTablet.compact(MajorCompactionReason.NORMAL) > 0) {
-        lss.accept(simuTablet.getNumFiles());
-      }
-
-      assertTrue(simuTablet.getTotalRead() < 6 * simuTablet.getTotalAdded());
-      assertTrue(lss.getAverage() < (n >= 8 ? 15 : 7));
-    }
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.java
deleted file mode 100644
index d956b61..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeLimitCompactionStrategyTest.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.tserver.compaction;
-
-import static org.apache.accumulo.tserver.compaction.DefaultCompactionStrategyTest.getServerContext;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("removal")
-public class SizeLimitCompactionStrategyTest {
-
-  private static Map<StoredTabletFile,DataFileValue> nfl(String... sa) {
-
-    HashMap<StoredTabletFile,DataFileValue> ret = new HashMap<>();
-    for (int i = 0; i < sa.length; i += 2) {
-      ret.put(new StoredTabletFile("hdfs://nn1/accumulo/tables/5/t-0001/" + sa[i]),
-          new DataFileValue(ConfigurationTypeHelper.getFixedMemoryAsBytes(sa[i + 1]), 1));
-    }
-
-    return ret;
-  }
-
-  public static void testSizeLimit(String opt, CompactionStrategy slcs) throws IOException {
-    HashMap<String,String> opts = new HashMap<>();
-    opts.put(opt, "1G");
-
-    slcs.init(opts);
-
-    KeyExtent ke = new KeyExtent(TableId.of("0"), null, null);
-    MajorCompactionRequest mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL,
-        DefaultConfiguration.getInstance(), getServerContext());
-
-    mcr.setFiles(nfl("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G"));
-
-    assertFalse(slcs.shouldCompact(mcr));
-    assertEquals(0, slcs.getCompactionPlan(mcr).inputFiles.size());
-    assertEquals(4, mcr.getFiles().size());
-
-    mcr.setFiles(nfl("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G", "f5", "500M", "f6", "500M",
-        "f7", "500M", "f8", "500M"));
-
-    assertTrue(slcs.shouldCompact(mcr));
-    assertEquals(nfl("f5", "500M", "f6", "500M", "f7", "500M", "f8", "500M").keySet(),
-        new HashSet<>(slcs.getCompactionPlan(mcr).inputFiles));
-    assertEquals(8, mcr.getFiles().size());
-  }
-
-  @Test
-  public void testLimits() throws IOException {
-    SizeLimitCompactionStrategy slcs = new SizeLimitCompactionStrategy();
-
-    testSizeLimit(SizeLimitCompactionStrategy.SIZE_LIMIT_OPT, slcs);
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeWindowTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeWindowTest.java
deleted file mode 100644
index ddde864..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/SizeWindowTest.java
+++ /dev/null
@@ -1,113 +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.tserver.compaction;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy.SizeWindow;
-import org.junit.jupiter.api.Test;
-
-public class SizeWindowTest {
-
-  static class TestSizeWindow extends SizeWindow {
-    private static Map<StoredTabletFile,DataFileValue> convert(Map<String,Integer> testData) {
-      Map<StoredTabletFile,DataFileValue> files = new HashMap<>();
-      testData.forEach((k, v) -> {
-        files.put(new StoredTabletFile("hdfs://nn1/accumulo/tables/5/t-0001/" + k),
-            new DataFileValue(v, 0));
-      });
-      return files;
-    }
-
-    /**
-     * A constructor that is more friendly for testing.
-     */
-    TestSizeWindow(Map<String,Integer> testData) {
-      super(convert(testData));
-    }
-  }
-
-  static Collection<String> getFileNames(SizeWindow sw) {
-    return sw.getFiles().stream().map(TabletFile::getFileName).collect(Collectors.toSet());
-  }
-
-  static Map<String,Integer> genTestData(int start, int end) {
-    Map<String,Integer> testData = new HashMap<>();
-
-    for (int i = start; i <= end; i++) {
-      testData.put("F" + i, i);
-    }
-
-    return testData;
-  }
-
-  @Test
-  public void testSlide() {
-
-    Map<String,Integer> testData = genTestData(1, 20);
-
-    TestSizeWindow tsw = new TestSizeWindow(testData);
-
-    SizeWindow tail = tsw.tail(10);
-
-    for (int i = 10; i <= 20; i++) {
-      int expectedSum = i * (i + 1) / 2 - ((i - 10) * (i - 9) / 2);
-      assertEquals(expectedSum, tail.sum());
-      assertEquals(10, tail.size());
-      assertEquals(genTestData(i - 9, i).keySet(), getFileNames(tail));
-      assertEquals(i, tail.topSize());
-      if (tail.slideUp()) {
-        assertTrue(i < 20);
-      } else {
-        assertEquals(20, i);
-      }
-    }
-  }
-
-  @Test
-  public void testPop() {
-    Map<String,Integer> testData = genTestData(1, 20);
-
-    TestSizeWindow tsw = new TestSizeWindow(testData);
-
-    SizeWindow tail = tsw.tail(10);
-
-    int expectedSize = 10;
-
-    while (expectedSize > 0) {
-      int expectedSum = expectedSize * (expectedSize + 1) / 2;
-      assertEquals(expectedSum, tail.sum());
-      assertEquals(expectedSize, tail.size());
-      assertEquals(genTestData(1, expectedSize).keySet(), getFileNames(tail));
-      assertEquals(expectedSize, tail.topSize());
-
-      tail.pop();
-      expectedSize--;
-    }
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/BasicCompactionStrategyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/BasicCompactionStrategyTest.java
deleted file mode 100644
index 5bf673c..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/compaction/strategies/BasicCompactionStrategyTest.java
+++ /dev/null
@@ -1,154 +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.tserver.compaction.strategies;
-
-import static org.apache.accumulo.tserver.compaction.DefaultCompactionStrategyTest.getServerContext;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.tserver.InMemoryMapTest;
-import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-import org.apache.accumulo.tserver.compaction.SizeLimitCompactionStrategyTest;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-/**
- * Tests org.apache.accumulo.tserver.compaction.BasicCompactionStrategy
- */
-@SuppressWarnings("removal")
-public class BasicCompactionStrategyTest {
-  private String largeCompressionType = "gz";
-  private BasicCompactionStrategy ttcs = null;
-  private MajorCompactionRequest mcr = null;
-  private AccumuloConfiguration conf = null;
-  private HashMap<String,String> opts = new HashMap<>();
-
-  private Map<StoredTabletFile,DataFileValue> createFileMap(String... sa) {
-
-    HashMap<StoredTabletFile,DataFileValue> ret = new HashMap<>();
-    for (int i = 0; i < sa.length; i += 2) {
-      ret.put(new StoredTabletFile("hdfs://nn1/accumulo/tables/5/t-0001/" + sa[i]),
-          new DataFileValue(ConfigurationTypeHelper.getFixedMemoryAsBytes(sa[i + 1]), 1));
-    }
-
-    return ret;
-  }
-
-  @BeforeEach
-  public void setup() {
-    opts.put(BasicCompactionStrategy.LARGE_FILE_COMPRESSION_TYPE, largeCompressionType);
-    opts.put(BasicCompactionStrategy.LARGE_FILE_COMPRESSION_THRESHOLD, "500M");
-    ttcs = new BasicCompactionStrategy();
-  }
-
-  @Test
-  public void testDefaultCompaction() {
-    ttcs.init(opts);
-    conf = DefaultConfiguration.getInstance();
-    KeyExtent ke = new KeyExtent(TableId.of("0"), null, null);
-    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf,
-        InMemoryMapTest.getServerContext());
-    Map<StoredTabletFile,DataFileValue> fileMap = createFileMap("f1", "10M", "f2", "10M", "f3",
-        "10M", "f4", "10M", "f5", "100M", "f6", "100M", "f7", "100M", "f8", "100M");
-    mcr.setFiles(fileMap);
-
-    assertTrue(ttcs.shouldCompact(mcr));
-    assertEquals(8, mcr.getFiles().size());
-
-    List<StoredTabletFile> filesToCompact = ttcs.getCompactionPlan(mcr).inputFiles;
-    assertEquals(fileMap.keySet(), new HashSet<>(filesToCompact));
-    assertEquals(8, filesToCompact.size());
-    assertNull(ttcs.getCompactionPlan(mcr).writeParameters);
-  }
-
-  @Test
-  public void testLargeCompaction() {
-    ttcs.init(opts);
-    conf = DefaultConfiguration.getInstance();
-    KeyExtent ke = new KeyExtent(TableId.of("0"), null, null);
-    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf, getServerContext());
-    Map<StoredTabletFile,DataFileValue> fileMap =
-        createFileMap("f1", "2G", "f2", "2G", "f3", "2G", "f4", "2G");
-    mcr.setFiles(fileMap);
-
-    assertTrue(ttcs.shouldCompact(mcr));
-    assertEquals(4, mcr.getFiles().size());
-
-    List<StoredTabletFile> filesToCompact = ttcs.getCompactionPlan(mcr).inputFiles;
-    assertEquals(fileMap.keySet(), new HashSet<>(filesToCompact));
-    assertEquals(4, filesToCompact.size());
-    assertEquals(largeCompressionType,
-        ttcs.getCompactionPlan(mcr).writeParameters.getCompressType());
-  }
-
-  @Test
-  public void testMissingType() {
-    opts.remove(BasicCompactionStrategy.LARGE_FILE_COMPRESSION_TYPE);
-    assertThrows(IllegalArgumentException.class, () -> ttcs.init(opts));
-  }
-
-  @Test
-  public void testMissingThreshold() {
-    opts.remove(BasicCompactionStrategy.LARGE_FILE_COMPRESSION_THRESHOLD);
-    assertThrows(IllegalArgumentException.class, () -> ttcs.init(opts));
-  }
-
-  @Test
-  public void testFileSubsetCompaction() {
-    ttcs.init(opts);
-    conf = DefaultConfiguration.getInstance();
-    KeyExtent ke = new KeyExtent(TableId.of("0"), null, null);
-    mcr = new MajorCompactionRequest(ke, MajorCompactionReason.NORMAL, conf, getServerContext());
-    Map<StoredTabletFile,DataFileValue> fileMap = createFileMap("f1", "1G", "f2", "10M", "f3",
-        "10M", "f4", "10M", "f5", "10M", "f6", "10M", "f7", "10M");
-    Map<StoredTabletFile,DataFileValue> filesToCompactMap =
-        createFileMap("f2", "10M", "f3", "10M", "f4", "10M", "f5", "10M", "f6", "10M", "f7", "10M");
-    mcr.setFiles(fileMap);
-
-    assertTrue(ttcs.shouldCompact(mcr));
-    assertEquals(7, mcr.getFiles().size());
-
-    List<StoredTabletFile> filesToCompact = ttcs.getCompactionPlan(mcr).inputFiles;
-    assertEquals(filesToCompactMap.keySet(), new HashSet<>(filesToCompact));
-    assertEquals(6, filesToCompact.size());
-    assertNull(ttcs.getCompactionPlan(mcr).writeParameters);
-  }
-
-  @Test
-  public void testLimits() throws IOException {
-    BasicCompactionStrategy slcs = new BasicCompactionStrategy();
-    SizeLimitCompactionStrategyTest.testSizeLimit(BasicCompactionStrategy.SIZE_LIMIT_OPT, slcs);
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java
index 5d30b06..3a9a773 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/DfsLoggerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.tserver.log;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.util.ArrayList;
@@ -68,6 +69,14 @@
     assertEquals(Durability.SYNC, chooseDurabilityForGroupCommit(lst));
   }
 
+  @Test
+  public void headerLengthTest() {
+    byte[] magic4 = DfsLogger.LOG_FILE_HEADER_V4.getBytes(UTF_8);
+    byte[] magic3 = DfsLogger.LOG_FILE_HEADER_V3.getBytes(UTF_8);
+
+    assertEquals(magic3.length, magic4.length, "Always expect log file headers to be same length");
+  }
+
   static Durability chooseDurabilityForGroupCommit(Collection<TabletMutations> mutations) {
     Durability result = Durability.NONE;
     for (TabletMutations tabletMutations : mutations) {
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java
deleted file mode 100644
index 99aa7a2..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystemTest.java
+++ /dev/null
@@ -1,545 +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.tserver.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-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.replication.ReplicationTarget;
-import org.apache.accumulo.core.replication.thrift.ReplicationServicer.Client;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.server.data.ServerMutation;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.tserver.logger.LogEvents;
-import org.apache.accumulo.tserver.logger.LogFileKey;
-import org.apache.accumulo.tserver.logger.LogFileValue;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class AccumuloReplicaSystemTest {
-
-  @Test
-  public void onlyChooseMutationsForDesiredTableWithOpenStatus() throws Exception {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-
-    LogFileKey key = new LogFileKey();
-    LogFileValue value = new LogFileValue();
-
-    // What is seq used for?
-    key.seq = 1L;
-
-    /*
-     * Disclaimer: the following series of LogFileKey and LogFileValue pairs have *no* bearing
-     * whatsoever in reality regarding what these entries would actually look like in a WAL. They
-     * are solely for testing that each LogEvents is handled, order is not important.
-     */
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("1"), null, null);
-    key.tabletId = 1;
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("2"), null, null);
-    key.tabletId = 2;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.OPEN;
-    key.tabletId = LogFileKey.VERSION;
-    key.tserverSession = "foobar";
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("badrow")));
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.COMPACTION_START;
-    key.tabletId = 2;
-    key.filename = "/accumulo/tables/1/t-000001/A000001.rf";
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("1"), null, null);
-    key.tabletId = 3;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.COMPACTION_FINISH;
-    key.tabletId = 6;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.tabletId = 3;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    dos.close();
-
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_NAME.getKey(), "source");
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ars.setConf(conf);
-
-    Status status =
-        Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(false).build();
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
-    WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", TableId.of("1")), dis,
-        new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, new HashSet<>());
-
-    // We stopped because we got to the end of the file
-    assertEquals(9, repl.entriesConsumed);
-    assertEquals(2, repl.walEdits.getEditsSize());
-    assertEquals(2, repl.sizeInRecords);
-    assertNotEquals(0, repl.sizeInBytes);
-  }
-
-  @Test
-  public void onlyChooseMutationsForDesiredTableWithClosedStatus() throws Exception {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-
-    LogFileKey key = new LogFileKey();
-    LogFileValue value = new LogFileValue();
-
-    // What is seq used for?
-    key.seq = 1L;
-
-    /*
-     * Disclaimer: the following series of LogFileKey and LogFileValue pairs have *no* bearing
-     * whatsoever in reality regarding what these entries would actually look like in a WAL. They
-     * are solely for testing that each LogEvents is handled, order is not important.
-     */
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("1"), null, null);
-    key.tabletId = 1;
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("2"), null, null);
-    key.tabletId = 2;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.OPEN;
-    key.tabletId = LogFileKey.VERSION;
-    key.tserverSession = "foobar";
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("badrow")));
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.COMPACTION_START;
-    key.tabletId = 2;
-    key.filename = "/accumulo/tables/1/t-000001/A000001.rf";
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("1"), null, null);
-    key.tabletId = 3;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.COMPACTION_FINISH;
-    key.tabletId = 6;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.tabletId = 3;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    dos.close();
-
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_NAME.getKey(), "source");
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ars.setConf(conf);
-
-    // Setting the file to be closed with the infinite end implies that we need to bump the begin up
-    // to Long.MAX_VALUE
-    // If it were still open, more data could be appended that we need to process
-    Status status =
-        Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).build();
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
-    WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", TableId.of("1")), dis,
-        new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, new HashSet<>());
-
-    // We stopped because we got to the end of the file
-    assertEquals(Long.MAX_VALUE, repl.entriesConsumed);
-    assertEquals(2, repl.walEdits.getEditsSize());
-    assertEquals(2, repl.sizeInRecords);
-    assertNotEquals(0, repl.sizeInBytes);
-  }
-
-  @Test
-  public void mutationsNotReReplicatedToPeers() throws Exception {
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_NAME.getKey(), "source");
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    ars.setConf(conf);
-
-    LogFileValue value = new LogFileValue();
-    value.mutations = new ArrayList<>();
-
-    Mutation m = new Mutation("row");
-    m.put("", "", new Value());
-    value.mutations.add(m);
-
-    m = new Mutation("row2");
-    m.put("", "", new Value());
-    m.addReplicationSource("peer");
-    value.mutations.add(m);
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(baos);
-
-    // Replicate our 2 mutations to "peer", from tableid 1 to tableid 1
-    ars.writeValueAvoidingReplicationCycles(out, value,
-        new ReplicationTarget("peer", "1", TableId.of("1")));
-
-    out.close();
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    DataInputStream in = new DataInputStream(bais);
-
-    int numMutations = in.readInt();
-    assertEquals(1, numMutations);
-
-    m = new Mutation();
-    m.readFields(in);
-
-    assertEquals("row", new String(m.getRow()));
-    assertEquals(1, m.getReplicationSources().size());
-    assertTrue(m.getReplicationSources().contains("source"),
-        "Expected source cluster to be listed in mutation replication source");
-  }
-
-  @Test
-  public void endOfFileExceptionOnClosedWalImpliesFullyReplicated() throws Exception {
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_NAME.getKey(), "source");
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ars.setConf(conf);
-
-    // Setting the file to be closed with the infinite end implies that we need to bump the begin up
-    // to Long.MAX_VALUE
-    // If it were still open, more data could be appended that we need to process
-    Status status =
-        Status.newBuilder().setBegin(100).setEnd(0).setInfiniteEnd(true).setClosed(true).build();
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(new byte[0]));
-    WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", TableId.of("1")), dis,
-        new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, new HashSet<>());
-
-    // We stopped because we got to the end of the file
-    assertEquals(Long.MAX_VALUE, repl.entriesConsumed);
-    assertEquals(0, repl.walEdits.getEditsSize());
-    assertEquals(0, repl.sizeInRecords);
-    assertEquals(0, repl.sizeInBytes);
-  }
-
-  @Test
-  public void endOfFileExceptionOnOpenWalImpliesMoreReplication() throws Exception {
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_NAME.getKey(), "source");
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ars.setConf(conf);
-
-    // Setting the file to be closed with the infinite end implies that we need to bump the begin up
-    // to Long.MAX_VALUE
-    // If it were still open, more data could be appended that we need to process
-    Status status =
-        Status.newBuilder().setBegin(100).setEnd(0).setInfiniteEnd(true).setClosed(false).build();
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(new byte[0]));
-    WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", TableId.of("1")), dis,
-        new Path("/accumulo/wals/tserver+port/wal"), status, Long.MAX_VALUE, new HashSet<>());
-
-    // We stopped because we got to the end of the file
-    assertEquals(0, repl.entriesConsumed);
-    assertEquals(0, repl.walEdits.getEditsSize());
-    assertEquals(0, repl.sizeInRecords);
-    assertEquals(0, repl.sizeInBytes);
-  }
-
-  @Test
-  public void restartInFileKnowsAboutPreviousTableDefines() throws Exception {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-
-    LogFileKey key = new LogFileKey();
-    LogFileValue value = new LogFileValue();
-
-    // What is seq used for?
-    key.seq = 1L;
-
-    /*
-     * Disclaimer: the following series of LogFileKey and LogFileValue pairs have *no* bearing
-     * whatsoever in reality regarding what these entries would actually look like in a WAL. They
-     * are solely for testing that each LogEvents is handled, order is not important.
-     */
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of("1"), null, null);
-    key.tabletId = 1;
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.tabletId = 1;
-    key.filename = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    dos.close();
-
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_NAME.getKey(), "source");
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ars.setConf(conf);
-
-    Status status =
-        Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(false).build();
-    DataInputStream dis = new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
-
-    HashSet<Integer> tids = new HashSet<>();
-
-    // Only consume the first mutation, not the second
-    WalReplication repl = ars.getWalEdits(new ReplicationTarget("peer", "1", TableId.of("1")), dis,
-        new Path("/accumulo/wals/tserver+port/wal"), status, 1L, tids);
-
-    // We stopped because we got to the end of the file
-    assertEquals(2, repl.entriesConsumed);
-    assertEquals(1, repl.walEdits.getEditsSize());
-    assertEquals(1, repl.sizeInRecords);
-    assertNotEquals(0, repl.sizeInBytes);
-
-    status = Status.newBuilder(status).setBegin(2).build();
-
-    // Consume the rest of the mutations
-    repl = ars.getWalEdits(new ReplicationTarget("peer", "1", TableId.of("1")), dis,
-        new Path("/accumulo/wals/tserver+port/wal"), status, 1L, tids);
-
-    // We stopped because we got to the end of the file
-    assertEquals(1, repl.entriesConsumed);
-    assertEquals(1, repl.walEdits.getEditsSize());
-    assertEquals(1, repl.sizeInRecords);
-    assertNotEquals(0, repl.sizeInBytes);
-  }
-
-  @Test
-  public void dontSendEmptyDataToPeer() throws Exception {
-    Client replClient = createMock(Client.class);
-    AccumuloReplicaSystem ars = createMock(AccumuloReplicaSystem.class);
-    WalEdits edits = new WalEdits(Collections.emptyList());
-    WalReplication walReplication = new WalReplication(edits, 0, 0, 0);
-
-    ReplicationTarget target = new ReplicationTarget("peer", "2", TableId.of("1"));
-    DataInputStream input = null;
-    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID());
-    Status status = null;
-    long sizeLimit = Long.MAX_VALUE;
-    String remoteTableId = target.getRemoteIdentifier();
-    TCredentials tcreds = null;
-    Set<Integer> tids = new HashSet<>();
-
-    WalClientExecReturn walClientExec = new WalClientExecReturn(ars, target, input, p, status,
-        sizeLimit, remoteTableId, tcreds, tids);
-
-    expect(ars.getWalEdits(target, input, p, status, sizeLimit, tids)).andReturn(walReplication);
-
-    replay(replClient, ars);
-
-    ReplicationStats stats = walClientExec.execute(replClient);
-
-    verify(replClient, ars);
-
-    assertEquals(new ReplicationStats(0L, 0L, 0L), stats);
-  }
-
-  @Test
-  public void consumedButNotSentDataShouldBeRecorded() throws Exception {
-    Client replClient = createMock(Client.class);
-    AccumuloReplicaSystem ars = createMock(AccumuloReplicaSystem.class);
-    WalEdits edits = new WalEdits(Collections.emptyList());
-    WalReplication walReplication = new WalReplication(edits, 0, 5, 0);
-
-    ReplicationTarget target = new ReplicationTarget("peer", "2", TableId.of("1"));
-    DataInputStream input = null;
-    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID());
-    Status status = null;
-    long sizeLimit = Long.MAX_VALUE;
-    String remoteTableId = target.getRemoteIdentifier();
-    TCredentials tcreds = null;
-    Set<Integer> tids = new HashSet<>();
-
-    WalClientExecReturn walClientExec = new WalClientExecReturn(ars, target, input, p, status,
-        sizeLimit, remoteTableId, tcreds, tids);
-
-    expect(ars.getWalEdits(target, input, p, status, sizeLimit, tids)).andReturn(walReplication);
-
-    replay(replClient, ars);
-
-    ReplicationStats stats = walClientExec.execute(replClient);
-
-    verify(replClient, ars);
-
-    assertEquals(new ReplicationStats(0L, 0L, 5L), stats);
-  }
-
-  @Test
-  public void testUserPassword() {
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ReplicationTarget target = new ReplicationTarget("peer", "peer_table", TableId.of("1"));
-    String user = "user", password = "password";
-
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_PEER_USER.getKey() + target.getPeerName(), user);
-    confMap.put(Property.REPLICATION_PEER_PASSWORD.getKey() + target.getPeerName(), password);
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    assertEquals(user, ars.getPrincipal(conf, target));
-    assertEquals(password, ars.getPassword(conf, target));
-  }
-
-  @Test
-  public void testUserKeytab() {
-    AccumuloReplicaSystem ars = new AccumuloReplicaSystem();
-    ReplicationTarget target = new ReplicationTarget("peer", "peer_table", TableId.of("1"));
-    String user = "user", keytab = "/etc/security/keytabs/replication.keytab";
-
-    Map<String,String> confMap = new HashMap<>();
-    confMap.put(Property.REPLICATION_PEER_USER.getKey() + target.getPeerName(), user);
-    confMap.put(Property.REPLICATION_PEER_KEYTAB.getKey() + target.getPeerName(), keytab);
-    AccumuloConfiguration conf = new ConfigurationCopy(confMap);
-
-    assertEquals(user, ars.getPrincipal(conf, target));
-    assertEquals(keytab, ars.getKeytab(conf, target));
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java
deleted file mode 100644
index cfe5fa4..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java
+++ /dev/null
@@ -1,204 +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.tserver.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.dataImpl.thrift.TMutation;
-import org.apache.accumulo.core.replication.thrift.WalEdits;
-import org.apache.accumulo.server.data.ServerMutation;
-import org.apache.accumulo.tserver.logger.LogEvents;
-import org.apache.accumulo.tserver.logger.LogFileKey;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Lists;
-
-@Deprecated
-public class BatchWriterReplicationReplayerTest {
-
-  private ClientContext context;
-  private AccumuloConfiguration conf;
-  private BatchWriter bw;
-
-  @BeforeEach
-  public void setUpContext() {
-    conf = createMock(AccumuloConfiguration.class);
-    bw = createMock(BatchWriter.class);
-    context = createMock(ClientContext.class);
-    expect(context.getConfiguration()).andReturn(conf).anyTimes();
-  }
-
-  @AfterEach
-  public void verifyMock() {
-    verify(context, conf, bw);
-  }
-
-  @Test
-  public void systemTimestampsAreSetOnUpdates() throws Exception {
-    final BatchWriterReplicationReplayer replayer = new BatchWriterReplicationReplayer();
-    final String tableName = "foo";
-    final long systemTimestamp = 1000;
-    final BatchWriterConfig bwCfg = new BatchWriterConfig();
-    bwCfg.setMaxMemory(1L);
-
-    LogFileKey key = new LogFileKey();
-    key.event = LogEvents.MANY_MUTATIONS;
-    key.seq = 1;
-    key.tabletId = 1;
-
-    WalEdits edits = new WalEdits();
-
-    // Make a mutation without timestamps
-    Mutation m = new Mutation("row");
-    m.put("cf", "cq1", "value");
-    m.put("cf", "cq2", "value");
-    m.put("cf", "cq3", "value");
-    m.put("cf", "cq4", "value");
-    m.put("cf", "cq5", "value");
-
-    // Make it a TMutation
-    TMutation tMutation = m.toThrift();
-
-    // And then make a ServerMutation from the TMutation, adding in our systemTimestamp
-    ServerMutation sMutation = new ServerMutation(tMutation);
-    sMutation.setSystemTimestamp(systemTimestamp);
-
-    // Serialize the ServerMutation (what AccumuloReplicaSystem will be doing)
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(baos);
-
-    key.write(out);
-    out.writeInt(1);
-    sMutation.write(out);
-
-    out.close();
-
-    // Add it to our "input" to BatchWriterReplicationReplayer
-    edits.addToEdits(ByteBuffer.wrap(baos.toByteArray()));
-
-    Mutation expectedMutation = new Mutation("row");
-    expectedMutation.put("cf", "cq1", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq2", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq3", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq4", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq5", sMutation.getSystemTimestamp(), "value");
-
-    expect(conf.getAsBytes(Property.TSERV_REPLICATION_BW_REPLAYER_MEMORY))
-        .andReturn(bwCfg.getMaxMemory());
-    expect(context.createBatchWriter(tableName, bwCfg)).andReturn(bw);
-
-    bw.addMutations(Lists.newArrayList(expectedMutation));
-    expectLastCall().once();
-
-    bw.close();
-    expectLastCall().once();
-
-    replay(context, conf, bw);
-
-    replayer.replicateLog(context, tableName, edits);
-  }
-
-  @Test
-  public void replicationSourcesArePreserved() throws Exception {
-    final BatchWriterReplicationReplayer replayer = new BatchWriterReplicationReplayer();
-    final String tableName = "foo";
-    final long systemTimestamp = 1000;
-    final String peerName = "peer";
-    final BatchWriterConfig bwCfg = new BatchWriterConfig();
-    bwCfg.setMaxMemory(1L);
-
-    LogFileKey key = new LogFileKey();
-    key.event = LogEvents.MANY_MUTATIONS;
-    key.seq = 1;
-    key.tabletId = 1;
-
-    WalEdits edits = new WalEdits();
-
-    // Make a mutation without timestamps
-    Mutation m = new Mutation("row");
-    m.put("cf", "cq1", "value");
-    m.put("cf", "cq2", "value");
-    m.put("cf", "cq3", "value");
-    m.put("cf", "cq4", "value");
-    m.put("cf", "cq5", "value");
-
-    // This Mutation "came" from a system called "peer"
-    m.addReplicationSource(peerName);
-
-    // Make it a TMutation
-    TMutation tMutation = m.toThrift();
-
-    // And then make a ServerMutation from the TMutation, adding in our systemTimestamp
-    ServerMutation sMutation = new ServerMutation(tMutation);
-    sMutation.setSystemTimestamp(systemTimestamp);
-
-    // Serialize the ServerMutation (what AccumuloReplicaSystem will be doing)
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(baos);
-
-    key.write(out);
-    out.writeInt(1);
-    sMutation.write(out);
-
-    out.close();
-
-    // Add it to our "input" to BatchWriterReplicationReplayer
-    edits.addToEdits(ByteBuffer.wrap(baos.toByteArray()));
-
-    Mutation expectedMutation = new Mutation("row");
-    expectedMutation.put("cf", "cq1", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq2", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq3", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq4", sMutation.getSystemTimestamp(), "value");
-    expectedMutation.put("cf", "cq5", sMutation.getSystemTimestamp(), "value");
-
-    // We expect our peer name to be preserved in the mutation that gets written
-    expectedMutation.addReplicationSource(peerName);
-
-    expect(conf.getAsBytes(Property.TSERV_REPLICATION_BW_REPLAYER_MEMORY))
-        .andReturn(bwCfg.getMaxMemory());
-    expect(context.createBatchWriter(tableName, bwCfg)).andReturn(bw);
-
-    bw.addMutations(Lists.newArrayList(expectedMutation));
-    expectLastCall().once();
-
-    bw.close();
-    expectLastCall().once();
-
-    replay(context, conf, bw);
-
-    replayer.replicateLog(context, tableName, edits);
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java
deleted file mode 100644
index 44d0b14..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/ReplicationProcessorTest.java
+++ /dev/null
@@ -1,102 +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.tserver.replication;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.createMockBuilder;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.ReplicaSystem;
-import org.apache.accumulo.server.replication.ReplicaSystemHelper;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.junit.jupiter.api.Test;
-
-@Deprecated
-public class ReplicationProcessorTest {
-
-  @Test
-  public void peerTypeExtractionFromConfiguration() {
-    ServerContext context = createMock(ServerContext.class);
-
-    String peerName = "peer";
-    String configuration = "java.lang.String,foo";
-    var conf = new ConfigurationCopy(Map.of(Property.REPLICATION_PEERS + peerName, configuration));
-    expect(context.getConfiguration()).andReturn(conf);
-    replay(context);
-
-    ReplicationProcessor proc = new ReplicationProcessor(context);
-    assertEquals(configuration, proc.getPeerType(peerName));
-    verify(context);
-  }
-
-  @Test
-  public void noPeerConfigurationThrowsAnException() {
-    ServerContext context = createMock(ServerContext.class);
-
-    var conf = new ConfigurationCopy(Map.of());
-    expect(context.getConfiguration()).andReturn(conf);
-    replay(context);
-
-    ReplicationProcessor proc = new ReplicationProcessor(context);
-    assertThrows(IllegalArgumentException.class, () -> proc.getPeerType("foo"));
-    verify(context);
-  }
-
-  @Test
-  public void filesWhichMakeNoProgressArentReplicatedAgain() throws Exception {
-    ReplicaSystem replica = createMock(ReplicaSystem.class);
-    ReplicaSystemHelper helper = createMock(ReplicaSystemHelper.class);
-    ReplicationProcessor proc = createMockBuilder(ReplicationProcessor.class)
-        .addMockedMethods("getReplicaSystem", "doesFileExist", "getStatus", "getHelper")
-        .createMock();
-
-    ReplicationTarget target = new ReplicationTarget("peer", "1", TableId.of("1"));
-    Status status =
-        Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).build();
-    Path path = new Path("/accumulo");
-
-    String queueKey = DistributedWorkQueueWorkAssignerHelper.getQueueKey(path.toString(), target);
-
-    expect(proc.getReplicaSystem(target)).andReturn(replica);
-    expect(proc.getStatus(path.toString(), target)).andReturn(status);
-    expect(proc.doesFileExist(path, target)).andReturn(true);
-    expect(proc.getHelper()).andReturn(helper);
-    expect(replica.replicate(path, status, target, helper)).andReturn(status);
-
-    replay(replica, proc);
-
-    proc.process(queueKey, path.toString().getBytes(UTF_8));
-
-    verify(replica, proc);
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java
index 8214823..35d5119 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplFileManagerTest.java
@@ -18,14 +18,12 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import static org.apache.accumulo.core.spi.compaction.CompactionKind.CHOP;
-import static org.apache.accumulo.core.spi.compaction.CompactionKind.SELECTOR;
 import static org.apache.accumulo.core.spi.compaction.CompactionKind.SYSTEM;
 import static org.apache.accumulo.core.spi.compaction.CompactionKind.USER;
+import static org.apache.accumulo.core.util.compaction.DeprecatedCompactionKind.SELECTOR;
 import static org.apache.accumulo.tserver.tablet.CompactableImplFileManagerTest.TestFileManager.SELECTION_EXPIRATION;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
@@ -44,9 +42,8 @@
 import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
 import org.apache.accumulo.core.spi.compaction.CompactionJob;
 import org.apache.accumulo.core.spi.compaction.CompactionKind;
-import org.apache.accumulo.tserver.tablet.CompactableImpl.ChopSelectionStatus;
-import org.apache.accumulo.tserver.tablet.CompactableImpl.FileManager.ChopSelector;
 import org.apache.accumulo.tserver.tablet.CompactableImpl.FileSelectionStatus;
+import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Sets;
@@ -60,17 +57,16 @@
     var tabletFiles = newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf");
 
     assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR);
+        fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, USER, SELECTOR);
 
     var job1 = newJob(CompactionKind.SYSTEM, "F00000.rf", "F00001.rf");
 
     assertTrue(fileMgr.reserveFiles(job1));
     assertEquals(job1.getSTFiles(), fileMgr.getCompactingFiles());
 
-    assertEquals(newFiles("F00002.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR);
+    assertEquals(newFiles("F00002.rf", "F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, USER, SELECTOR);
 
     var job2 = newJob(CompactionKind.SYSTEM, "F00002.rf", "F00003.rf");
 
@@ -80,8 +76,8 @@
     // try to reserve files reserved by other compactions, should fail
     assertFalse(fileMgr.reserveFiles(newJob(CompactionKind.SYSTEM, "F00001.rf", "F00002.rf")));
 
-    assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CompactionKind.SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR);
+    assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CompactionKind.SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, USER, SELECTOR);
 
     fileMgr.completed(job2, newFile("C00004.rf"));
     fileMgr.completed(job1, newFile("C00005.rf"));
@@ -89,13 +85,13 @@
     tabletFiles = newFiles("C00004.rf", "C00005.rf");
 
     assertEquals(newFiles("C00004.rf", "C00005.rf"),
-        fileMgr.getCandidates(tabletFiles, CompactionKind.SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR);
+        fileMgr.getCandidates(tabletFiles, CompactionKind.SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, USER, SELECTOR);
 
     var job3 = newJob(CompactionKind.SYSTEM, "C00004.rf", "C00005.rf");
     assertTrue(fileMgr.reserveFiles(job3));
 
-    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR);
+    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, USER, SELECTOR);
 
     fileMgr.completed(job3, newFile("A00006.rf"));
     assertEquals(Set.of(), fileMgr.getCompactingFiles());
@@ -115,29 +111,29 @@
     assertFalse(fileMgr.reserveFiles(staleJob));
 
     assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, USER, SELECTOR);
+        fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, USER, SELECTOR);
 
     assertTrue(fileMgr.initiateSelection(USER));
 
     assertFalse(fileMgr.reserveFiles(staleJob));
 
-    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR);
+    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, USER, SELECTOR);
 
     assertTrue(fileMgr.beginSelection());
 
     assertFalse(fileMgr.reserveFiles(staleJob));
 
-    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, USER, SELECTOR);
+    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, USER, SELECTOR);
 
     fileMgr.finishSelection(newFiles("F00000.rf", "F00001.rf", "F00002.rf"), false);
 
     assertFalse(fileMgr.reserveFiles(staleJob));
 
-    assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false));
+    assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM));
     assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf"),
-        fileMgr.getCandidates(tabletFiles, USER, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR);
+        fileMgr.getCandidates(tabletFiles, USER));
+    assertNoCandidates(fileMgr, tabletFiles, SELECTOR);
 
     assertFalse(fileMgr.reserveFiles(newJob(SELECTOR, "F00000.rf", "F00001.rf")));
     assertFalse(fileMgr.reserveFiles(newJob(SYSTEM, "F00000.rf", "F00001.rf")));
@@ -155,19 +151,18 @@
     // reservation was successfully made
     fileMgr.setNanoTime(2 * SELECTION_EXPIRATION.toNanos());
 
-    assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertEquals(newFiles("F00002.rf"), fileMgr.getCandidates(tabletFiles, USER, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR);
+    assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertEquals(newFiles("F00002.rf"), fileMgr.getCandidates(tabletFiles, USER));
+    assertNoCandidates(fileMgr, tabletFiles, SELECTOR);
 
     fileMgr.completed(job1, newFile("C00004.rf"));
     assertEquals(FileSelectionStatus.RESERVED, fileMgr.getSelectionStatus());
 
     tabletFiles = newFiles("C00004.rf", "F00002.rf", "F00003.rf");
 
-    assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertEquals(newFiles("F00002.rf", "C00004.rf"),
-        fileMgr.getCandidates(tabletFiles, USER, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR);
+    assertEquals(newFiles("F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertEquals(newFiles("F00002.rf", "C00004.rf"), fileMgr.getCandidates(tabletFiles, USER));
+    assertNoCandidates(fileMgr, tabletFiles, SELECTOR);
 
     var job2 = newJob(USER, "F00002.rf", "C00004.rf");
 
@@ -181,9 +176,8 @@
 
     tabletFiles = newFiles("C00005.rf", "F00003.rf");
 
-    assertEquals(newFiles("C00005.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, USER, CHOP, SELECTOR);
+    assertEquals(newFiles("C00005.rf", "F00003.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, USER, SELECTOR);
   }
 
   @Test
@@ -200,8 +194,8 @@
     assertFalse(fileMgr.reserveFiles(newJob(SYSTEM, "F00001.rf", "F00002.rf")));
 
     assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, USER, false));
-    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, CHOP, SELECTOR);
+        fileMgr.getCandidates(tabletFiles, USER));
+    assertNoCandidates(fileMgr, tabletFiles, SYSTEM, SELECTOR);
 
     // advance time to a point where the selection is eligible to expire
     fileMgr.setNanoTime(2 * SELECTION_EXPIRATION.toNanos());
@@ -209,10 +203,10 @@
     // now that the selection is eligible to expire, the selected files should be available as
     // system compaction candidates
     assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, USER, false));
+        fileMgr.getCandidates(tabletFiles, USER));
     assertEquals(newFiles("F00000.rf", "F00001.rf", "F00002.rf", "F00003.rf"),
-        fileMgr.getCandidates(tabletFiles, SYSTEM, false));
-    assertNoCandidates(fileMgr, tabletFiles, CHOP, SELECTOR);
+        fileMgr.getCandidates(tabletFiles, SYSTEM));
+    assertNoCandidates(fileMgr, tabletFiles, SELECTOR);
 
     // a system compaction should be able to reserve selected files after expiration which should
     // deactivate the selection
@@ -353,14 +347,14 @@
     assertEquals(Sets.union(job1.getSTFiles(), job2.getSTFiles()), fileMgr.getCompactingFiles());
 
     // all tablet files are selected, so there are no candidates for system compaction
-    assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, SYSTEM, false));
+    assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, SYSTEM));
 
     // this time when the user compaction is canceled jobs are running, so transition to canceled
     fileMgr.userCompactionCanceled();
     assertEquals(FileSelectionStatus.CANCELED, fileMgr.getSelectionStatus());
 
     // files that were selected should now be available as candidates after canceling
-    assertEquals(newFiles("F00004.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM, false));
+    assertEquals(newFiles("F00004.rf"), fileMgr.getCandidates(tabletFiles, SYSTEM));
 
     // when this job completes it should not transition from CANCELED to NOT_ACTIVE because there is
     // still another
@@ -378,65 +372,10 @@
 
   }
 
-  @Test
-  public void testChop() {
-    TestFileManager fileMgr = new TestFileManager();
-
-    // simulate a compaction because files that were created by compaction are remembered as not
-    // needing a chop
-    var job1 = newJob(SYSTEM, "F00000.rf", "F00001.rf");
-    assertTrue(fileMgr.reserveFiles(job1));
-    fileMgr.completed(job1, newFile("C00005.rf"));
-
-    var tabletFiles = newFiles("C00005.rf", "F00002.rf", "F00003.rf", "F00004.rf");
-
-    ChopSelector chopSel = fileMgr.initiateChop(tabletFiles);
-    assertEquals(ChopSelectionStatus.SELECTING, fileMgr.getChopStatus());
-
-    assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CHOP, false));
-
-    // this should not include C00005.rf because it was created by a compaction observed by the file
-    // manager
-    assertEquals(newFiles("F00002.rf", "F00003.rf", "F00004.rf"), chopSel.getFilesToExamine());
-
-    chopSel.selectChopFiles(newFiles("F00002.rf", "F00004.rf"));
-    assertEquals(ChopSelectionStatus.SELECTED, fileMgr.getChopStatus());
-
-    assertEquals(newFiles("F00002.rf", "F00004.rf"),
-        fileMgr.getCandidates(tabletFiles, CHOP, false));
-
-    // simulate compacting one of the files that needs to be chopped, but this should not finish the
-    // chop because more files need to be chopped
-    var job2 = newJob(CHOP, "F00002.rf");
-    assertTrue(fileMgr.reserveFiles(job2));
-    fileMgr.completed(job2, newFile("C00006.rf"));
-    tabletFiles = newFiles("C00004.rf", "C00006.rf", "F00003.rf", "F00004.rf");
-    assertFalse(fileMgr.finishChop(tabletFiles));
-    assertEquals(ChopSelectionStatus.SELECTED, fileMgr.getChopStatus());
-    assertThrows(IllegalStateException.class, fileMgr::finishMarkingChop);
-
-    assertEquals(newFiles("F00004.rf"), fileMgr.getCandidates(tabletFiles, CHOP, false));
-
-    // simulate compacting the last file to chop. should cause the chop finish
-    var job3 = newJob(CHOP, "F00004.rf");
-    assertTrue(fileMgr.reserveFiles(job3));
-    fileMgr.completed(job3, newFile("C00007.rf"));
-    tabletFiles = newFiles("C00004.rf", "C00006.rf", "F00003.rf", "C00007.rf");
-    assertTrue(fileMgr.finishChop(tabletFiles));
-
-    assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, CHOP, false));
-    assertEquals(ChopSelectionStatus.MARKING, fileMgr.getChopStatus());
-    assertEquals(Set.of(), fileMgr.getCompactingFiles());
-
-    fileMgr.finishMarkingChop();
-    assertEquals(ChopSelectionStatus.NOT_ACTIVE, fileMgr.getChopStatus());
-
-  }
-
   private void assertNoCandidates(TestFileManager fileMgr, Set<StoredTabletFile> tabletFiles,
       CompactionKind... kinds) {
     for (CompactionKind kind : kinds) {
-      assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, kind, false));
+      assertEquals(Set.of(), fileMgr.getCandidates(tabletFiles, kind));
     }
 
   }
@@ -481,7 +420,7 @@
   }
 
   static StoredTabletFile newFile(String f) {
-    return new StoredTabletFile("hdfs://nn1/accumulo/tables/1/t-0001/" + f);
+    return StoredTabletFile.of(new Path("hdfs://nn1/accumulo/tables/1/t-0001/" + f));
   }
 
   static Set<StoredTabletFile> newFiles(String... strings) {
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplTest.java
index 4551f64..cddd2e1 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactableImplTest.java
@@ -30,12 +30,12 @@
 import java.util.UUID;
 
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
 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.accumulo.core.util.compaction.DeprecatedCompactionKind;
 import org.junit.jupiter.api.Test;
 
 public class CompactableImplTest {
@@ -50,13 +50,13 @@
       Set<StoredTabletFile> nextFiles, CompactionKind kind, boolean propagateDeletes,
       boolean initiallySelectedAll, Long compactionId) {
 
-    TabletFile compactTmpName = newFile("C00000A.rf_tmp");
+    StoredTabletFile compactTmpName = newFile("C00000A.rf_tmp");
     String compactorId = "cid";
     short priority = 9;
     CompactionExecutorId ceid = CompactionExecutorIdImpl.externalId("ecs1");
 
-    return new ExternalCompactionMetadata(jobFiles, nextFiles, compactTmpName, compactorId, kind,
-        priority, ceid, propagateDeletes, initiallySelectedAll, compactionId);
+    return new ExternalCompactionMetadata(jobFiles, nextFiles, compactTmpName.getTabletFile(),
+        compactorId, kind, priority, ceid, propagateDeletes, initiallySelectedAll, compactionId);
   }
 
   ExternalCompactionId newEcid() {
@@ -306,8 +306,8 @@
     var ecm1 = newECM(fileSet1, newFiles("F00003", "F00004"), CompactionKind.USER, true, true);
 
     var fileSet2 = newFiles("F00003", "F00004");
-    var ecm2 =
-        newECM(fileSet2, newFiles("F00001", "F00002"), CompactionKind.SELECTOR, true, true, null);
+    var ecm2 = newECM(fileSet2, newFiles("F00001", "F00002"), DeprecatedCompactionKind.SELECTOR,
+        true, true, null);
 
     var ecid1 = newEcid();
     var ecid2 = newEcid();
@@ -327,8 +327,8 @@
   @Test
   public void testSelectorWithCompactionId() {
     var fileSet1 = newFiles("F00001", "F00002");
-    var ecm1 =
-        newECM(fileSet1, newFiles("F00003", "F00004"), CompactionKind.SELECTOR, true, true, 5L);
+    var ecm1 = newECM(fileSet1, newFiles("F00003", "F00004"), DeprecatedCompactionKind.SELECTOR,
+        true, true, 5L);
 
     var ecid1 = newEcid();
 
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/TabletTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/TabletTest.java
deleted file mode 100644
index 692272c..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/TabletTest.java
+++ /dev/null
@@ -1,66 +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.tserver.tablet;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Map;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.WriteParameters;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("removal")
-public class TabletTest {
-
-  @Test
-  public void correctValuesSetForProperties() {
-
-    CompactionPlan plan = EasyMock.createMock(CompactionPlan.class);
-    WriteParameters writeParams = EasyMock.createMock(WriteParameters.class);
-    plan.writeParameters = writeParams;
-
-    long hdfsBlockSize = 10000L, blockSize = 5000L, indexBlockSize = 500L;
-    int replication = 5;
-    String compressType = "snappy";
-
-    EasyMock.expect(writeParams.getHdfsBlockSize()).andReturn(hdfsBlockSize).times(2);
-    EasyMock.expect(writeParams.getBlockSize()).andReturn(blockSize).times(2);
-    EasyMock.expect(writeParams.getIndexBlockSize()).andReturn(indexBlockSize).times(2);
-    EasyMock.expect(writeParams.getCompressType()).andReturn(compressType).times(2);
-    EasyMock.expect(writeParams.getReplication()).andReturn(replication).times(2);
-
-    EasyMock.replay(plan, writeParams);
-
-    Map<String,String> aConf = CompactableUtils.computeOverrides(writeParams);
-
-    EasyMock.verify(plan, writeParams);
-
-    assertEquals(hdfsBlockSize, Long.parseLong(aConf.get(Property.TABLE_FILE_BLOCK_SIZE.getKey())));
-    assertEquals(blockSize,
-        Long.parseLong(aConf.get(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE.getKey())));
-    assertEquals(indexBlockSize,
-        Long.parseLong(aConf.get(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX.getKey())));
-    assertEquals(compressType, aConf.get(Property.TABLE_FILE_COMPRESSION_TYPE.getKey()));
-    assertEquals(replication,
-        Integer.parseInt(aConf.get(Property.TABLE_FILE_REPLICATION.getKey())));
-  }
-}
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/util/CreateEmptyTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/util/CreateEmptyTest.java
new file mode 100644
index 0000000..ad9f73f
--- /dev/null
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/util/CreateEmptyTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.tserver.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.tserver.log.DfsLogger.LOG_FILE_HEADER_V4;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.mock;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.client.rfile.RFile;
+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.crypto.CryptoEnvironmentImpl;
+import org.apache.accumulo.core.crypto.CryptoUtils;
+import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
+import org.apache.accumulo.core.spi.crypto.CryptoService;
+import org.apache.accumulo.core.spi.crypto.GenericCryptoServiceFactory;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.accumulo.tserver.log.DfsLogger;
+import org.apache.accumulo.tserver.logger.LogEvents;
+import org.apache.accumulo.tserver.logger.LogFileKey;
+import org.apache.accumulo.tserver.logger.LogFileValue;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class CreateEmptyTest {
+  @TempDir
+  private static File tempDir;
+
+  private ServerContext context;
+
+  @BeforeEach
+  public void init() throws IOException {
+    ConfigurationCopy config = new ConfigurationCopy(DefaultConfiguration.getInstance());
+    config.set(Property.INSTANCE_VOLUMES.getKey(), "file:///");
+
+    context = mock(ServerContext.class);
+    expect(context.getCryptoFactory()).andReturn(new GenericCryptoServiceFactory()).anyTimes();
+    expect(context.getConfiguration()).andReturn(config).anyTimes();
+    expect(context.getHadoopConf()).andReturn(new Configuration()).anyTimes();
+    VolumeManager volumeManager = VolumeManagerImpl.get(config, new Configuration());
+    expect(context.getVolumeManager()).andReturn(volumeManager).anyTimes();
+    replay(context);
+  }
+
+  @AfterEach
+  public void verifyMock() {
+    verify(context);
+  }
+
+  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
+  @Test
+  public void exceptionOnFileExistsTest() throws Exception {
+    CreateEmpty createEmpty = new CreateEmpty();
+
+    String wal1 = genFilename(tempDir.getAbsolutePath() + "/empty", ".wal");
+    String rf1 = genFilename(tempDir.getAbsolutePath() + "/empty", ".rf");
+
+    // create the file so it exists
+    File f = new File(wal1);
+    assertTrue(f.createNewFile());
+
+    String[] walArgs = {"--type", "WAL", wal1};
+    CreateEmpty.Opts walOpts = new CreateEmpty.Opts();
+    walOpts.parseArgs("accumulo create-empty", walArgs);
+
+    assertThrows(IllegalArgumentException.class,
+        () -> createEmpty.createEmptyWal(walOpts, context));
+
+    // create the file so it exists
+    File f2 = new File(rf1);
+    assertTrue(f2.createNewFile());
+
+    String[] rfArgs = {"--type", "RF", rf1};
+    CreateEmpty.Opts rfOpts = new CreateEmpty.Opts();
+    rfOpts.parseArgs("accumulo create-empty", rfArgs);
+    assertThrows(IllegalArgumentException.class,
+        () -> createEmpty.createEmptyRFile(walOpts, context));
+  }
+
+  @Test
+  public void createRfileTest() throws Exception {
+    CreateEmpty createEmpty = new CreateEmpty();
+
+    String file1 = genFilename(tempDir.getAbsolutePath() + "/empty", ".rf");
+    String file2 = genFilename(tempDir.getAbsolutePath() + "/empty", ".rf");
+
+    String[] args = {"--type", "RF", file1, file2};
+    CreateEmpty.Opts opts = new CreateEmpty.Opts();
+    opts.parseArgs("accumulo create-empty", args);
+
+    createEmpty.createEmptyRFile(opts, context);
+    VolumeManager vm = context.getVolumeManager();
+    assertTrue(vm.exists(new Path(file1)));
+    try (var scanner = RFile.newScanner().from(file1).build()) {
+      assertEquals(0, scanner.stream().count());
+    }
+
+    assertTrue(vm.exists(new Path(file2)));
+    try (var scanner = RFile.newScanner().from(file2).build()) {
+      assertEquals(0, scanner.stream().count());
+    }
+
+  }
+
+  /**
+   * Validate that the default type is RF (RecoveryWithEmptyRFileIT also needs this(
+   */
+  @Test
+  public void createRfileDefaultTest() throws Exception {
+    CreateEmpty createEmpty = new CreateEmpty();
+
+    String file1 = genFilename(tempDir.getAbsolutePath() + "/empty", ".rf");
+
+    String[] args = {file1};
+    CreateEmpty.Opts opts = new CreateEmpty.Opts();
+    opts.parseArgs("accumulo create-empty", args);
+
+    createEmpty.createEmptyRFile(opts, context);
+    VolumeManager vm = context.getVolumeManager();
+    assertTrue(vm.exists(new Path(file1)));
+    try (var scanner = RFile.newScanner().from(file1).build()) {
+      assertEquals(0, scanner.stream().count());
+    }
+  }
+
+  @Test
+  public void createWalTest() throws Exception {
+    CreateEmpty createEmpty = new CreateEmpty();
+
+    String file1 = genFilename(tempDir.getAbsolutePath() + "/empty", ".wal");
+    String file2 = genFilename(tempDir.getAbsolutePath() + "/empty", ".wal");
+
+    String[] args = {"--type", "WAL", file1, file2};
+    CreateEmpty.Opts opts = new CreateEmpty.Opts();
+    opts.parseArgs("accumulo create-empty", args);
+
+    createEmpty.createEmptyWal(opts, context);
+
+    checkWalContext(file1);
+    readLogFile(file1);
+
+    checkWalContext(file2);
+  }
+
+  /**
+   * Reads the log file and looks for specific information (crypto id, event == OPEN)
+   */
+  private void checkWalContext(final String expected) throws IOException {
+    Path path = new Path(expected);
+    VolumeManager vm = context.getVolumeManager();
+    assertTrue(vm.exists(path));
+
+    vm.open(path);
+    try (InputStream inputStream = vm.open(path).getWrappedStream();
+        DataInputStream dis = new DataInputStream(inputStream)) {
+      byte[] headerBuf = new byte[1024];
+      int len = dis.read(headerBuf, 0, LOG_FILE_HEADER_V4.length());
+      assertEquals(LOG_FILE_HEADER_V4.length(), len);
+      assertEquals(LOG_FILE_HEADER_V4,
+          new String(headerBuf, 0, LOG_FILE_HEADER_V4.length(), UTF_8));
+
+      CryptoEnvironment env = new CryptoEnvironmentImpl(CryptoEnvironment.Scope.WAL);
+      CryptoService cryptoService = context.getCryptoFactory().getService(env,
+          context.getConfiguration().getAllCryptoProperties());
+
+      byte[] decryptionParams = cryptoService.getFileEncrypter(env).getDecryptionParameters();
+
+      var cryptParams = CryptoUtils.readParams(dis);
+      assertArrayEquals(decryptionParams, cryptParams);
+
+      LogFileKey key = new LogFileKey();
+      key.readFields(dis);
+
+      assertEquals(key.event, LogEvents.OPEN);
+      assertEquals("", key.tserverSession);
+      assertNull(key.filename);
+    }
+  }
+
+  /**
+   * Scan through log file and check that there is one event.
+   */
+  private void readLogFile(final String filename) throws Exception {
+    Path path = new Path(filename);
+    LogFileKey key = new LogFileKey();
+    LogFileValue value = new LogFileValue();
+
+    FileSystem fs = context.getVolumeManager().getFileSystemByPath(path);
+
+    CryptoEnvironment env = new CryptoEnvironmentImpl(CryptoEnvironment.Scope.WAL);
+    CryptoService cryptoService = context.getCryptoFactory().getService(env,
+        context.getConfiguration().getAllCryptoProperties());
+
+    int eventCount = 0;
+    try (final FSDataInputStream fsinput = fs.open(path);
+        DataInputStream input = DfsLogger.getDecryptingStream(fsinput, cryptoService)) {
+      while (true) {
+        try {
+          key.readFields(input);
+          value.readFields(input);
+        } catch (EOFException ex) {
+          break;
+        }
+        eventCount++;
+      }
+    } catch (DfsLogger.LogHeaderIncompleteException e) {
+      fail("Could not read header for {}" + path);
+    } finally {
+      // empty wal has 1 event (OPEN)
+      assertEquals(1, eventCount);
+    }
+  }
+
+  // tempDir is per test suite - generate a one-up count file for each call.
+  private static final AtomicInteger fileCount = new AtomicInteger(0);
+
+  private String genFilename(final String prefix, final String extension) {
+    return prefix + fileCount.incrementAndGet() + extension;
+  }
+}
diff --git a/server/tserver/src/test/resources/log4j2-test.properties b/server/tserver/src/test/resources/log4j2-test.properties
index 410e99a..15c1a8a 100644
--- a/server/tserver/src/test/resources/log4j2-test.properties
+++ b/server/tserver/src/test/resources/log4j2-test.properties
@@ -39,9 +39,6 @@
 logger.04.name = org.apache.accumulo.core.util.shell.Shell
 logger.04.level = fatal
 
-logger.05.name = org.apache.commons.vfs2.impl.DefaultFileSystemManager
-logger.05.level = warn
-
 logger.06.name = org.apache.hadoop.io.compress.CodecPool
 logger.06.level = warn
 
@@ -66,15 +63,11 @@
 logger.13.name = org.apache.accumulo.core.file.rfile.bcfile
 logger.13.level = info
 
-logger.14.name = org.apache.accumulo.server.util.ReplicationTableUtil
-logger.14.level = trace
+logger.14.name = org.apache.accumulo.core.clientImpl.TabletServerBatchReaderIterator
+logger.14.level = info
 
-logger.15.name = org.apache.accumulo.core.clientImpl.TabletServerBatchReaderIterator
+logger.15.name = org.apache.accumulo.core.clientImpl.ThriftScanner
 logger.15.level = info
 
-logger.16.name = org.apache.accumulo.core.clientImpl.ThriftScanner
-logger.16.level = info
-
 rootLogger.level = info
 rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/shell/pom.xml b/shell/pom.xml
index ecf7bc4..e91d031 100644
--- a/shell/pom.xml
+++ b/shell/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-shell</artifactId>
   <name>Apache Accumulo Shell</name>
@@ -40,10 +40,6 @@
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>com.google.code.gson</groupId>
-      <artifactId>gson</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
@@ -84,10 +80,6 @@
       <artifactId>libthrift</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.jline</groupId>
       <artifactId>jline</artifactId>
     </dependency>
diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 615b2cc..6396277 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -67,7 +67,7 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.thrift.TConstraintViolationSummary;
-import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
+import org.apache.accumulo.core.tabletingest.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.accumulo.core.util.format.Formatter;
@@ -106,7 +106,6 @@
 import org.apache.accumulo.shell.commands.ExitCommand;
 import org.apache.accumulo.shell.commands.ExportTableCommand;
 import org.apache.accumulo.shell.commands.ExtensionCommand;
-import org.apache.accumulo.shell.commands.FateCommand;
 import org.apache.accumulo.shell.commands.FlushCommand;
 import org.apache.accumulo.shell.commands.FormatterCommand;
 import org.apache.accumulo.shell.commands.GetAuthsCommand;
@@ -261,8 +260,7 @@
     this.writer = terminal.writer();
   }
 
-  // this is visible only for FateCommandTest, otherwise, should be private or inline
-  protected boolean authenticateUser(AccumuloClient client, AuthenticationToken token)
+  private boolean authenticateUser(AccumuloClient client, AuthenticationToken token)
       throws AccumuloException, AccumuloSecurityException {
     return client.securityOperations().authenticateUser(client.whoami(), token);
   }
@@ -334,9 +332,6 @@
       return false;
     }
 
-    if (options.isDebugEnabled()) {
-      log.warn("Configure debugging through your logging configuration file");
-    }
     authTimeout = TimeUnit.MINUTES.toNanos(options.getAuthTimeout());
     disableAuthTimeout = options.isAuthTimeoutDisabled();
 
@@ -396,34 +391,26 @@
 
     rootToken = new Token();
 
-    @SuppressWarnings("deprecation")
     Command[] dataCommands = {new DeleteCommand(), new DeleteManyCommand(), new DeleteRowsCommand(),
-        new EGrepCommand(), new FormatterCommand(),
-        new org.apache.accumulo.shell.commands.InterpreterCommand(), new GrepCommand(),
-        new ImportDirectoryCommand(), new InsertCommand(), new MaxRowCommand(), new ScanCommand()};
-    @SuppressWarnings("deprecation")
+        new EGrepCommand(), new FormatterCommand(), new GrepCommand(), new ImportDirectoryCommand(),
+        new InsertCommand(), new MaxRowCommand(), new ScanCommand()};
     Command[] debuggingCommands =
-        {new ClasspathCommand(), new org.apache.accumulo.shell.commands.DebugCommand(),
-            new ListScansCommand(), new ListCompactionsCommand(), new TraceCommand(),
-            new PingCommand(), new ListBulkCommand(), new ListTabletsCommand()};
-    @SuppressWarnings("deprecation")
-    Command[] execCommands = {new ExecfileCommand(), new HistoryCommand(), new ExtensionCommand(),
-        new org.apache.accumulo.shell.commands.ScriptCommand()};
+        {new ClasspathCommand(), new ListScansCommand(), new ListCompactionsCommand(),
+            new TraceCommand(), new PingCommand(), new ListBulkCommand(), new ListTabletsCommand()};
+    Command[] execCommands = {new ExecfileCommand(), new HistoryCommand(), new ExtensionCommand()};
     Command[] exitCommands = {new ByeCommand(), new ExitCommand(), new QuitCommand()};
     Command[] helpCommands =
         {new AboutCommand(), new HelpCommand(), new InfoCommand(), new QuestionCommand()};
-    @SuppressWarnings("deprecation")
-    Command[] iteratorCommands = {new DeleteIterCommand(),
-        new org.apache.accumulo.shell.commands.DeleteScanIterCommand(), new ListIterCommand(),
-        new SetIterCommand(), new org.apache.accumulo.shell.commands.SetScanIterCommand(),
-        new SetShellIterCommand(), new ListShellIterCommand(), new DeleteShellIterCommand()};
+    Command[] iteratorCommands =
+        {new DeleteIterCommand(), new ListIterCommand(), new SetIterCommand(),
+            new SetShellIterCommand(), new ListShellIterCommand(), new DeleteShellIterCommand()};
     Command[] otherCommands = {new HiddenCommand()};
     Command[] permissionsCommands = {new GrantCommand(), new RevokeCommand(),
         new SystemPermissionsCommand(), new TablePermissionsCommand(), new UserPermissionsCommand(),
         new NamespacePermissionsCommand()};
-    Command[] stateCommands = {new AuthenticateCommand(), new ClsCommand(), new ClearCommand(),
-        new FateCommand(), new NoTableCommand(), new SleepCommand(), new TableCommand(),
-        new UserCommand(), new WhoAmICommand()};
+    Command[] stateCommands =
+        {new AuthenticateCommand(), new ClsCommand(), new ClearCommand(), new NoTableCommand(),
+            new SleepCommand(), new TableCommand(), new UserCommand(), new WhoAmICommand()};
     Command[] tableCommands = {new CloneTableCommand(), new ConfigCommand(),
         new CreateTableCommand(), new DeleteTableCommand(), new DropTableCommand(), new DUCommand(),
         new ExportTableCommand(), new ImportTableCommand(), new OfflineCommand(),
@@ -485,7 +472,7 @@
     } else {
       throw new IllegalArgumentException("No table or namespace specified");
     }
-    String tableContext = getTableContextFromProps(tableProps);
+    String tableContext = tableProps.get(Property.TABLE_CLASSLOADER_CONTEXT.getKey());
 
     if (tableContext != null && !tableContext.isEmpty()) {
       ClassLoaderUtil.initContextFactory(new ConfigurationCopy(
@@ -494,30 +481,6 @@
     return ClassLoaderUtil.getClassLoader(tableContext);
   }
 
-  private static String getTableContextFromProps(Map<String,String> props) {
-    String tableContext = null;
-    for (Entry<String,String> entry : props.entrySet()) {
-      // look for either the old property or the new one, but
-      // if the new one is set, stop looking and let it take precedence
-      if (entry.getKey().equals(Property.TABLE_CLASSLOADER_CONTEXT.getKey())
-          && entry.getValue() != null && !entry.getValue().isEmpty()) {
-        return entry.getValue();
-      }
-      @SuppressWarnings("removal")
-      Property TABLE_CLASSPATH = Property.TABLE_CLASSPATH;
-      if (entry.getKey().equals(TABLE_CLASSPATH.getKey())) {
-        // don't return even if this is set; instead,
-        // keep looking, in case we find the newer property set
-        tableContext = entry.getValue();
-        if (tableContext != null && !tableContext.isEmpty()) {
-          log.warn("Deprecated table context property detected. '{}' should be replaced by '{}'",
-              TABLE_CLASSPATH.getKey(), Property.TABLE_CLASSLOADER_CONTEXT.getKey());
-        }
-      }
-    }
-    return tableContext;
-  }
-
   @Override
   public String keyword() {
     return "shell";
diff --git a/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java b/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
index 0b4822c..84a9347 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/ShellOptionsJC.java
@@ -60,10 +60,6 @@
       description = "disables tab completion (for less overhead when scripting)")
   private boolean tabCompletionDisabled;
 
-  @Parameter(names = "--debug", description = "enables client debugging"
-      + "; deprecated, configure debugging through your logging configuration file")
-  private boolean debugEnabled;
-
   @Parameter(names = {"-?", "--help"}, help = true, description = "display this help")
   private boolean helpEnabled;
 
@@ -152,10 +148,6 @@
     return tabCompletionDisabled;
   }
 
-  public boolean isDebugEnabled() {
-    return debugEnabled;
-  }
-
   public boolean isHelpEnabled() {
     return helpEnabled;
   }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/BulkImportListIterator.java b/shell/src/main/java/org/apache/accumulo/shell/commands/BulkImportListIterator.java
index 4846916..a1b443a 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/BulkImportListIterator.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/BulkImportListIterator.java
@@ -22,9 +22,9 @@
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.util.DurationFormat;
 
 public class BulkImportListIterator implements Iterator<String> {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ClasspathCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ClasspathCommand.java
index d3d6014..892aa5e 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ClasspathCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ClasspathCommand.java
@@ -18,21 +18,21 @@
  */
 package org.apache.accumulo.shell.commands;
 
+import java.io.File;
 import java.io.PrintWriter;
+import java.util.Arrays;
 
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
 
 public class ClasspathCommand extends Command {
-  @SuppressWarnings("deprecation")
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) {
 
     final PrintWriter writer = shellState.getWriter();
-    org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.printClassPath(s -> {
-      writer.print(s);
-    }, true);
+    printClassPath(writer);
     return 0;
   }
 
@@ -45,4 +45,16 @@
   public int numArgs() {
     return 0;
   }
+
+  public static void printClassPath(PrintWriter writer) {
+    writer.println("Accumulo Shell Classpath:");
+
+    final String javaClassPath = System.getProperty("java.class.path");
+    if (javaClassPath == null) {
+      throw new IllegalStateException("java.class.path is not set");
+    }
+    Arrays.stream(javaClassPath.split(File.pathSeparator)).forEach(writer::println);
+
+    writer.println();
+  }
 }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
index f7d1e28..9b09365 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/CompactCommand.java
@@ -30,6 +30,8 @@
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.PluginConfig;
 import org.apache.accumulo.core.compaction.CompactionSettings;
+import org.apache.accumulo.core.compaction.ShellCompactCommandConfigurer;
+import org.apache.accumulo.core.compaction.ShellCompactCommandSelector;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.ShellUtil;
 import org.apache.commons.cli.CommandLine;
@@ -37,7 +39,7 @@
 import org.apache.commons.cli.Options;
 
 public class CompactCommand extends TableOperation {
-  private Option noFlushOption, waitOpt, profileOpt, cancelOpt, strategyOpt, strategyConfigOpt;
+  private Option noFlushOption, waitOpt, profileOpt, cancelOpt;
 
   // file selection and file output options
   private Option enameOption, epathOption, sizeLtOption, sizeGtOption, minFilesOption,
@@ -117,23 +119,19 @@
     put(cl, sopts, copts, outIndexBlockSizeOpt, CompactionSettings.OUTPUT_INDEX_BLOCK_SIZE_OPT);
     put(cl, sopts, copts, outReplication, CompactionSettings.OUTPUT_REPLICATION_OPT);
 
-    if ((!sopts.isEmpty() || !copts.isEmpty()) && (cl.hasOption(strategyOpt.getOpt())
-        || cl.hasOption(selectorOpt.getLongOpt()) || cl.hasOption(configurerOpt.getLongOpt()))) {
+    if ((!sopts.isEmpty() || !copts.isEmpty())
+        && (cl.hasOption(selectorOpt.getLongOpt()) || cl.hasOption(configurerOpt.getLongOpt()))) {
       throw new IllegalArgumentException(
-          "Can not specify compaction strategy/selector/configurer with file selection and file output options.");
+          "Can not specify compaction selector/configurer with file selection and file output options.");
     }
 
     if (!sopts.isEmpty()) {
-      PluginConfig selectorCfg = new PluginConfig(
-          "org.apache.accumulo.tserver.compaction.strategies.ConfigurableCompactionStrategy",
-          sopts);
+      var selectorCfg = new PluginConfig(ShellCompactCommandSelector.class.getName(), sopts);
       compactionConfig.setSelector(selectorCfg);
     }
 
     if (!copts.isEmpty()) {
-      PluginConfig configurerConfig = new PluginConfig(
-          "org.apache.accumulo.tserver.compaction.strategies.ConfigurableCompactionStrategy",
-          copts);
+      var configurerConfig = new PluginConfig(ShellCompactCommandConfigurer.class.getName(), copts);
       compactionConfig.setConfigurer(configurerConfig);
     }
   }
@@ -172,23 +170,14 @@
 
     setupConfigurableCompaction(cl, compactionConfig);
 
-    if (cl.hasOption(strategyOpt.getOpt())) {
-      if (cl.hasOption(selectorOpt.getLongOpt()) || cl.hasOption(configurerOpt.getLongOpt())) {
-        throw new IllegalArgumentException(
-            "Can not specify a strategy with a selector or configurer");
-      }
-      configureCompactionStrat(cl);
-    } else {
-      if (cl.hasOption(selectorOpt.getLongOpt())) {
-        compactionConfig.setSelector(new PluginConfig(cl.getOptionValue(selectorOpt.getLongOpt()),
-            ShellUtil.parseMapOpt(cl, selectorConfigOpt)));
-      }
+    if (cl.hasOption(selectorOpt.getLongOpt())) {
+      compactionConfig.setSelector(new PluginConfig(cl.getOptionValue(selectorOpt.getLongOpt()),
+          ShellUtil.parseMapOpt(cl, selectorConfigOpt)));
+    }
 
-      if (cl.hasOption(configurerOpt.getLongOpt())) {
-        compactionConfig
-            .setConfigurer(new PluginConfig(cl.getOptionValue(configurerOpt.getLongOpt()),
-                ShellUtil.parseMapOpt(cl, configurerConfigOpt)));
-      }
+    if (cl.hasOption(configurerOpt.getLongOpt())) {
+      compactionConfig.setConfigurer(new PluginConfig(cl.getOptionValue(configurerOpt.getLongOpt()),
+          ShellUtil.parseMapOpt(cl, configurerConfigOpt)));
     }
 
     if (cl.hasOption(hintsOption.getLongOpt())) {
@@ -198,14 +187,6 @@
     return super.execute(fullCommand, cl, shellState);
   }
 
-  @SuppressWarnings("removal")
-  private void configureCompactionStrat(final CommandLine cl) {
-    var csc = new org.apache.accumulo.core.client.admin.CompactionStrategyConfig(
-        cl.getOptionValue(strategyOpt.getOpt()));
-    csc.setOptions(ShellUtil.parseMapOpt(cl, strategyConfigOpt));
-    compactionConfig.setCompactionStrategy(csc);
-  }
-
   private Option newLAO(String lopt, String desc) {
     return new Option(null, lopt, true, desc);
   }
@@ -226,12 +207,6 @@
     profileOpt.setArgName("profile");
     opts.addOption(profileOpt);
 
-    strategyOpt = new Option("s", "strategy", true, "compaction strategy class name");
-    opts.addOption(strategyOpt);
-    strategyConfigOpt = new Option("sc", "strategyConfig", true,
-        "Key value options for compaction strategy.  Expects <prop>=<value>{,<prop>=<value>}");
-    opts.addOption(strategyConfigOpt);
-
     hintsOption = newLAO("exec-hints",
         "Compaction execution hints.  Expects <prop>=<value>{,<prop>=<value>}");
     opts.addOption(hintsOption);
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DebugCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DebugCommand.java
deleted file mode 100644
index a68e04d..0000000
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/DebugCommand.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.shell.commands;
-
-import org.apache.accumulo.shell.Shell;
-import org.apache.accumulo.shell.Shell.Command;
-import org.apache.commons.cli.CommandLine;
-
-/**
- * @deprecated since 2.0; this command shouldn't be used; users should configure debug logging with
- *             their log configuration file instead
- */
-@Deprecated(since = "2.0.0")
-public class DebugCommand extends Command {
-  @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) {
-    shellState.printException(new IllegalArgumentException("The debug command is deprecated; "
-        + "configure debug logging through your log configuration file instead."));
-    return 1;
-  }
-
-  @Override
-  public String description() {
-    return "Deprecated since 2.0";
-  }
-
-  @Override
-  public String usage() {
-    return getName() + " [ on | off ] # this is now deprecated and does nothing";
-  }
-
-  @Override
-  public int numArgs() {
-    return Shell.NO_FIXED_ARG_LENGTH_CHECK;
-  }
-}
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteCommand.java
index b239a42..3491724 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteCommand.java
@@ -29,7 +29,7 @@
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
+import org.apache.accumulo.core.tabletingest.thrift.ConstraintViolationException;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteManyCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteManyCommand.java
index 5384736..f2c2105 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteManyCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteManyCommand.java
@@ -41,10 +41,6 @@
       throws Exception {
     final String tableName = OptUtil.getTableOpt(cl, shellState);
 
-    @SuppressWarnings("deprecation")
-    final org.apache.accumulo.core.util.interpret.ScanInterpreter interpeter =
-        getInterpreter(cl, tableName, shellState);
-
     // handle first argument, if present, the authorizations list to
     // scan with
     final Authorizations auths = getAuths(cl, shellState);
@@ -57,12 +53,12 @@
     addScanIterators(shellState, cl, scanner, tableName);
 
     // handle remaining optional arguments
-    scanner.setRange(getRange(cl, interpeter));
+    scanner.setRange(getRange(cl));
 
     scanner.setTimeout(getTimeout(cl), TimeUnit.MILLISECONDS);
 
     // handle columns
-    fetchColumns(cl, scanner, interpeter);
+    fetchColumns(cl, scanner);
 
     // output / delete the records
     final BatchWriter writer = shellState.getAccumuloClient().createBatchWriter(tableName,
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteScanIterCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteScanIterCommand.java
deleted file mode 100644
index 7e58774..0000000
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/DeleteScanIterCommand.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.shell.commands;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.shell.Shell;
-import org.apache.accumulo.shell.Shell.Command;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionGroup;
-import org.apache.commons.cli.Options;
-
-@Deprecated
-public class DeleteScanIterCommand extends Command {
-  private Option nameOpt, allOpt;
-
-  @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
-      throws Exception {
-    Shell.log.warn("Deprecated, use {}", new DeleteShellIterCommand().getName());
-    final String tableName = OptUtil.getTableOpt(cl, shellState);
-
-    if (cl.hasOption(allOpt.getOpt())) {
-      final List<IteratorSetting> tableScanIterators =
-          shellState.scanIteratorOptions.remove(tableName);
-      if (tableScanIterators == null) {
-        Shell.log.info("No scan iterators set on table {}", tableName);
-      } else {
-        Shell.log.info("Removed the following scan iterators from table {}:{}", tableName,
-            tableScanIterators);
-      }
-    } else if (cl.hasOption(nameOpt.getOpt())) {
-      final String name = cl.getOptionValue(nameOpt.getOpt());
-      final List<IteratorSetting> tableScanIterators =
-          shellState.scanIteratorOptions.get(tableName);
-      if (tableScanIterators != null) {
-        boolean found = false;
-        for (Iterator<IteratorSetting> iter = tableScanIterators.iterator(); iter.hasNext();) {
-          if (iter.next().getName().equals(name)) {
-            iter.remove();
-            found = true;
-            break;
-          }
-        }
-        if (found) {
-          Shell.log.info("Removed scan iterator {} from table {} ({} left)", name, tableName,
-              shellState.scanIteratorOptions.get(tableName).size());
-          if (shellState.scanIteratorOptions.get(tableName).isEmpty()) {
-            shellState.scanIteratorOptions.remove(tableName);
-          }
-        } else {
-          Shell.log.info("No iterator named {} found for table {}", name, tableName);
-        }
-      } else {
-        Shell.log.info("No iterator named {} found for table {}", name, tableName);
-      }
-    }
-
-    return 0;
-  }
-
-  @Override
-  public String description() {
-    return "(deprecated) deletes a table-specific scan iterator so it is no longer used"
-        + " during this shell session";
-  }
-
-  @Override
-  public Options getOptions() {
-    final Options o = new Options();
-
-    OptionGroup nameGroup = new OptionGroup();
-
-    nameOpt = new Option("n", "name", true, "iterator to delete");
-    nameOpt.setArgName("itername");
-
-    allOpt = new Option("a", "all", false, "delete all scan iterators");
-    allOpt.setArgName("all");
-
-    nameGroup.addOption(nameOpt);
-    nameGroup.addOption(allOpt);
-    nameGroup.setRequired(true);
-    o.addOptionGroup(nameGroup);
-    o.addOption(OptUtil.tableOpt("table to delete scan iterators from"));
-
-    return o;
-  }
-
-  @Override
-  public int numArgs() {
-    return 0;
-  }
-}
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
deleted file mode 100644
index 9ac7342..0000000
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
+++ /dev/null
@@ -1,374 +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.shell.commands;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.fate.FateTxId.parseTidFromUserInput;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.Base64;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Formatter;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-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.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.fate.AdminUtil;
-import org.apache.accumulo.core.fate.ReadOnlyRepo;
-import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
-import org.apache.accumulo.core.fate.Repo;
-import org.apache.accumulo.core.fate.ZooStore;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.util.FastFormat;
-import org.apache.accumulo.shell.Shell;
-import org.apache.accumulo.shell.Shell.Command;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionGroup;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.zookeeper.KeeperException;
-
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
-
-/**
- * Manage FATE transactions
- */
-public class FateCommand extends Command {
-  private static final String warning =
-      "WARNING: This command is deprecated for removal. Use 'accumulo admin'\n";
-
-  // this class serializes references to interfaces with the concrete class name
-  private static class InterfaceSerializer<T> implements JsonSerializer<T> {
-    @Override
-    public JsonElement serialize(T link, Type type, JsonSerializationContext context) {
-      JsonElement je = context.serialize(link, link.getClass());
-      JsonObject jo = new JsonObject();
-      jo.add(link.getClass().getName(), je);
-      return jo;
-    }
-  }
-
-  // the purpose of this class is to be serialized as JSon for display
-  public static class ByteArrayContainer {
-    public String asUtf8;
-    public String asBase64;
-
-    ByteArrayContainer(byte[] ba) {
-      asUtf8 = new String(ba, UTF_8);
-      asBase64 = Base64.getUrlEncoder().encodeToString(ba);
-    }
-  }
-
-  // serialize byte arrays in human and machine readable ways
-  private static class ByteArraySerializer implements JsonSerializer<byte[]> {
-    @Override
-    public JsonElement serialize(byte[] link, Type type, JsonSerializationContext context) {
-      return context.serialize(new ByteArrayContainer(link));
-    }
-  }
-
-  // the purpose of this class is to be serialized as JSon for display
-  public static class FateStack {
-    String txIdString;
-    List<ReadOnlyRepo<FateCommand>> stack;
-
-    FateStack(Long txid, List<ReadOnlyRepo<FateCommand>> stack) {
-      this.txIdString = FastFormat.toHexString(txid);
-      this.stack = stack;
-    }
-  }
-
-  private Option cancel;
-  private Option delete;
-  private Option dump;
-  private Option fail;
-  private Option list;
-  private Option print;
-  private Option summary;
-  private Option secretOption;
-  private Option statusOption;
-  private Option disablePaginationOpt;
-
-  protected String getZKRoot(ClientContext context) {
-    return context.getZooKeeperRoot();
-  }
-
-  synchronized ZooReaderWriter getZooReaderWriter(ClientContext context, String secret) {
-    if (secret == null) {
-      secret = SiteConfiguration.auto().get(Property.INSTANCE_SECRET);
-    }
-    return context.getZooReader().asWriter(secret);
-  }
-
-  protected ZooStore<FateCommand> getZooStore(String fateZkPath, ZooReaderWriter zrw)
-      throws KeeperException, InterruptedException {
-    return new ZooStore<>(fateZkPath, zrw);
-  }
-
-  @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
-      throws ParseException, KeeperException, InterruptedException, IOException, AccumuloException,
-      AccumuloSecurityException {
-    Shell.log.warn(warning);
-
-    ClientContext context = shellState.getContext();
-    boolean failedCommand = false;
-
-    AdminUtil<FateCommand> admin = new AdminUtil<>(false);
-
-    String zkRoot = getZKRoot(context);
-    String fatePath = zkRoot + Constants.ZFATE;
-    var managerLockPath = ServiceLock.path(zkRoot + Constants.ZMANAGER_LOCK);
-    var tableLocksPath = ServiceLock.path(zkRoot + Constants.ZTABLE_LOCKS);
-    ZooReaderWriter zk = getZooReaderWriter(context, cl.getOptionValue(secretOption.getOpt()));
-    ZooStore<FateCommand> zs = getZooStore(fatePath, zk);
-
-    if (cl.hasOption(cancel.getOpt())) {
-      String[] txids = cl.getOptionValues(cancel.getOpt());
-      validateArgs(txids);
-      throw new ParseException(
-          "Option not available. Use 'accumulo admin fate -c " + String.join(" ", txids) + "'");
-    } else if (cl.hasOption(fail.getOpt())) {
-      String[] txids = cl.getOptionValues(fail.getOpt());
-      validateArgs(txids);
-      failedCommand = failTx(shellState.getWriter(), admin, zs, zk, managerLockPath, txids);
-    } else if (cl.hasOption(delete.getOpt())) {
-      String[] txids = cl.getOptionValues(delete.getOpt());
-      validateArgs(txids);
-      failedCommand = deleteTx(shellState.getWriter(), admin, zs, zk, managerLockPath, txids);
-    } else if (cl.hasOption(list.getOpt())) {
-      printTx(shellState, admin, zs, zk, tableLocksPath, cl.getOptionValues(list.getOpt()), cl);
-    } else if (cl.hasOption(print.getOpt())) {
-      printTx(shellState, admin, zs, zk, tableLocksPath, cl.getOptionValues(print.getOpt()), cl);
-    } else if (cl.hasOption(summary.getOpt())) {
-      throw new ParseException("Option not available. Use 'accumulo admin fate --summary'");
-    } else if (cl.hasOption(dump.getOpt())) {
-      String output = dumpTx(zs, cl.getOptionValues(dump.getOpt()));
-      shellState.getWriter().println(output);
-    } else {
-      throw new ParseException("Invalid command option");
-    }
-
-    return failedCommand ? 1 : 0;
-  }
-
-  String dumpTx(ZooStore<FateCommand> zs, String[] args) {
-    List<Long> txids;
-    if (args.length == 1) {
-      txids = zs.list();
-    } else {
-      txids = new ArrayList<>();
-      for (int i = 1; i < args.length; i++) {
-        txids.add(parseTidFromUserInput(args[i]));
-      }
-    }
-
-    Gson gson = new GsonBuilder()
-        .registerTypeAdapter(ReadOnlyRepo.class, new InterfaceSerializer<>())
-        .registerTypeAdapter(Repo.class, new InterfaceSerializer<>())
-        .registerTypeAdapter(byte[].class, new ByteArraySerializer()).setPrettyPrinting().create();
-
-    List<FateStack> txStacks = new ArrayList<>();
-    for (Long txid : txids) {
-      List<ReadOnlyRepo<FateCommand>> repoStack = zs.getStack(txid);
-      txStacks.add(new FateStack(txid, repoStack));
-    }
-
-    return gson.toJson(txStacks);
-  }
-
-  protected void printTx(Shell shellState, AdminUtil<FateCommand> admin, ZooStore<FateCommand> zs,
-      ZooReaderWriter zk, ServiceLock.ServiceLockPath tableLocksPath, String[] args, CommandLine cl)
-      throws InterruptedException, KeeperException, IOException {
-    // Parse transaction ID filters for print display
-    Set<Long> filterTxid = new HashSet<>();
-    if (args != null && args.length >= 1) {
-      for (int i = 0; i < args.length; i++) {
-        if (!args[i].isEmpty()) {
-          Long val = parseTidFromUserInput(args[i]);
-          filterTxid.add(val);
-        }
-      }
-    }
-
-    // Parse TStatus filters for print display
-    EnumSet<TStatus> statusFilter = getCmdLineStatusFilters(cl);
-
-    StringBuilder buf = new StringBuilder(8096);
-    Formatter fmt = new Formatter(buf);
-    admin.print(zs, zk, tableLocksPath, fmt, filterTxid, statusFilter);
-    shellState.printLines(Collections.singletonList(buf.toString()).iterator(),
-        !cl.hasOption(disablePaginationOpt.getOpt()));
-  }
-
-  protected boolean deleteTx(PrintWriter out, AdminUtil<FateCommand> admin,
-      ZooStore<FateCommand> zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath, String[] args)
-      throws InterruptedException, KeeperException {
-    for (int i = 1; i < args.length; i++) {
-      if (admin.prepDelete(zs, zk, zLockManagerPath, args[i])) {
-        admin.deleteLocks(zk, zLockManagerPath, args[i]);
-      } else {
-        out.printf("Could not delete transaction: %s%n", args[i]);
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private void validateArgs(String[] args) throws ParseException {
-    if (args.length < 1) {
-      throw new ParseException("Must provide transaction ID");
-    }
-  }
-
-  public boolean failTx(PrintWriter out, AdminUtil<FateCommand> admin, ZooStore<FateCommand> zs,
-      ZooReaderWriter zk, ServiceLockPath managerLockPath, String[] args) {
-    boolean success = true;
-    for (int i = 1; i < args.length; i++) {
-      if (!admin.prepFail(zs, zk, managerLockPath, args[i])) {
-        out.printf("Could not fail transaction: %s%n", args[i]);
-        return !success;
-      }
-    }
-    return success;
-  }
-
-  @Override
-  public String description() {
-    return "manage FATE transactions";
-  }
-
-  @Override
-  public Options getOptions() {
-    final Options o = new Options();
-
-    OptionGroup commands = new OptionGroup();
-    cancel =
-        new Option("cancel", "cancel-submitted", true, "cancel new or submitted FaTE transactions");
-    cancel.setArgName("txid");
-    cancel.setArgs(Option.UNLIMITED_VALUES);
-    cancel.setOptionalArg(false);
-
-    fail = new Option("fail", "fail", true,
-        "Transition FaTE transaction status to FAILED_IN_PROGRESS (requires Manager to be down)");
-    fail.setArgName("txid");
-    fail.setArgs(Option.UNLIMITED_VALUES);
-    fail.setOptionalArg(false);
-
-    delete = new Option("delete", "delete", true,
-        "delete locks associated with FaTE transactions (requires Manager to be down)");
-    delete.setArgName("txid");
-    delete.setArgs(Option.UNLIMITED_VALUES);
-    delete.setOptionalArg(false);
-
-    list = new Option("list", "list", true,
-        "print FaTE transaction information. Filter on id(s) with FATE[id] or id list ");
-    list.setArgName("txid");
-    list.setArgs(Option.UNLIMITED_VALUES);
-    list.setOptionalArg(true);
-
-    print = new Option("print", "print", true,
-        "print FaTE transaction information. Filter on id(s) with FATE[id] or id list ");
-    print.setArgName("txid");
-    print.setArgs(Option.UNLIMITED_VALUES);
-    print.setOptionalArg(true);
-
-    summary =
-        new Option("summary", "summary", true, "print a summary of FaTE transaction information");
-    summary.setArgName("--json");
-    summary.setOptionalArg(true);
-
-    dump = new Option("dump", "dump", true, "dump FaTE transaction information details");
-    dump.setArgName("txid");
-    dump.setArgs(Option.UNLIMITED_VALUES);
-    dump.setOptionalArg(true);
-
-    commands.addOption(cancel);
-    commands.addOption(fail);
-    commands.addOption(delete);
-    commands.addOption(list);
-    commands.addOption(print);
-    commands.addOption(summary);
-    commands.addOption(dump);
-    o.addOptionGroup(commands);
-
-    secretOption = new Option("s", "secret", true, "specify the instance secret to use");
-    secretOption.setOptionalArg(false);
-    o.addOption(secretOption);
-    statusOption = new Option("t", "status-type", true,
-        "filter 'print' on the transaction status type(s) {NEW, SUBMITTED, IN_PROGRESS,"
-            + " FAILED_IN_PROGRESS, FAILED, SUCCESSFUL}");
-    statusOption.setArgs(Option.UNLIMITED_VALUES);
-    statusOption.setOptionalArg(false);
-    o.addOption(statusOption);
-    disablePaginationOpt =
-        new Option("np", "no-pagination", false, "disables pagination of output");
-    o.addOption(disablePaginationOpt);
-    return o;
-  }
-
-  @Override
-  public int numArgs() {
-    // Arg length varies between 1 to n
-    return -1;
-  }
-
-  @Override
-  public String usage() {
-    String msg = super.usage();
-    return warning + msg;
-  }
-
-  /**
-   * If provided on the command line, get the TStatus values provided.
-   *
-   * @param cl the command line
-   * @return a set of status filters, or an empty set if none provides
-   */
-  private EnumSet<TStatus> getCmdLineStatusFilters(CommandLine cl) {
-    EnumSet<TStatus> statusFilter = null;
-    if (cl.hasOption(statusOption.getOpt())) {
-      statusFilter = EnumSet.noneOf(TStatus.class);
-      String[] tstat = cl.getOptionValues(statusOption.getOpt());
-      for (String element : tstat) {
-        statusFilter.add(TStatus.valueOf(element));
-      }
-    }
-    return statusFilter;
-  }
-
-}
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/FormatterCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/FormatterCommand.java
index bf517f4..786b4a0 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/FormatterCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/FormatterCommand.java
@@ -18,19 +18,12 @@
  */
 package org.apache.accumulo.shell.commands;
 
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.format.Formatter;
 import org.apache.accumulo.shell.Shell;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
 
 public class FormatterCommand extends ShellPluginConfigurationCommand {
 
-  private Option interpeterOption;
-
   public FormatterCommand() {
     super("formatter", Property.TABLE_FORMATTER_CLASS, "f");
   }
@@ -46,36 +39,4 @@
         Property.TABLE_FORMATTER_CLASS);
   }
 
-  @Override
-  public Options getOptions() {
-    final Options options = super.getOptions();
-
-    interpeterOption = new Option("i", "interpeter", false, "configure class as interpreter also");
-
-    options.addOption(interpeterOption);
-
-    return options;
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  protected void setPlugin(final CommandLine cl, final Shell shellState, final String tableName,
-      final String className) throws AccumuloException, AccumuloSecurityException {
-    super.setPlugin(cl, shellState, tableName, className);
-    if (cl.hasOption(interpeterOption.getOpt())) {
-      shellState.getAccumuloClient().tableOperations().setProperty(tableName,
-          Property.TABLE_INTERPRETER_CLASS.toString(), className);
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  protected void removePlugin(final CommandLine cl, final Shell shellState, final String tableName)
-      throws AccumuloException, AccumuloSecurityException {
-    super.removePlugin(cl, shellState, tableName);
-    if (cl.hasOption(interpeterOption.getOpt())) {
-      shellState.getAccumuloClient().tableOperations().removeProperty(tableName,
-          Property.TABLE_INTERPRETER_CLASS.toString());
-    }
-  }
 }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java
index afe5536..be7bb42 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java
@@ -30,8 +30,7 @@
 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.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.TabletColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.TextUtil;
@@ -64,8 +63,8 @@
     try (PrintLine p =
         outputFile == null ? new PrintShell(shellState.getReader()) : new PrintFile(outputFile)) {
       if (verbose) {
-        String systemTableToCheck =
-            MetadataTable.NAME.equals(tableName) ? RootTable.NAME : MetadataTable.NAME;
+        String systemTableToCheck = AccumuloTable.METADATA.tableName().equals(tableName)
+            ? AccumuloTable.ROOT.tableName() : AccumuloTable.METADATA.tableName();
         final Scanner scanner =
             shellState.getAccumuloClient().createScanner(systemTableToCheck, Authorizations.EMPTY);
         TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/GrepCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/GrepCommand.java
index bc6e106..c43b034 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/GrepCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/GrepCommand.java
@@ -64,9 +64,6 @@
         }
       }
       final Class<? extends Formatter> formatter = getFormatter(cl, tableName, shellState);
-      @SuppressWarnings("deprecation")
-      final org.apache.accumulo.core.util.interpret.ScanInterpreter interpeter =
-          getInterpreter(cl, tableName, shellState);
 
       // handle first argument, if present, the authorizations list to
       // scan with
@@ -83,7 +80,7 @@
       final Authorizations auths = getAuths(cl, shellState);
       final BatchScanner scanner =
           shellState.getAccumuloClient().createBatchScanner(tableName, auths, numThreads);
-      scanner.setRanges(Collections.singletonList(getRange(cl, interpeter)));
+      scanner.setRanges(Collections.singletonList(getRange(cl)));
 
       scanner.setTimeout(getTimeout(cl), TimeUnit.MILLISECONDS);
 
@@ -98,7 +95,7 @@
       }
       try {
         // handle columns
-        fetchColumns(cl, scanner, interpeter);
+        fetchColumns(cl, scanner);
 
         // output the records
         printRecords(cl, shellState, config, scanner, formatter, printFile);
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/HiddenCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/HiddenCommand.java
index 761ec0a..1872ab5 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/HiddenCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/HiddenCommand.java
@@ -19,8 +19,8 @@
 package org.apache.accumulo.shell.commands;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
 import java.util.Base64;
 
 import org.apache.accumulo.shell.Shell;
@@ -31,7 +31,6 @@
 import org.jline.utils.InfoCmp;
 
 public class HiddenCommand extends Command {
-  private static final SecureRandom random = new SecureRandom();
 
   @Override
   public String description() {
@@ -41,7 +40,7 @@
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
       throws Exception {
-    if (random.nextInt(10) == 0) {
+    if (RANDOM.get().nextInt(10) == 0) {
       shellState.getTerminal().puts(InfoCmp.Capability.bell);
       shellState.getWriter().println();
       shellState.getWriter()
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ImportDirectoryCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ImportDirectoryCommand.java
index 2c5b9fa..80117d0 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ImportDirectoryCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ImportDirectoryCommand.java
@@ -41,11 +41,9 @@
         + " sets the time. If the -i ignore option is supplied then no exception will be thrown"
         + " when attempting to import files from an empty source directory. An info log message"
         + " will be displayed indicating the source directory is empty, but no error is thrown.\n"
-        + " Passing 3 arguments will use the old bulk import. The new bulk import only takes"
-        + " 2 arguments:  <directory> true|false";
+        + " Bulk import only takes 2 arguments:  <directory> true|false";
   }
 
-  @SuppressWarnings("deprecation")
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
       throws IOException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
@@ -66,21 +64,9 @@
             .tableTime(setTime).ignoreEmptyDir(ignore).load();
         break;
       }
-      case 3: {
-        // warn using deprecated bulk import
-        Shell.log.warn(
-            "Deprecated since 2.0.0. New bulk import technique does not take a failure directory "
-                + "as an argument.");
-        String failureDir = args[1];
-        setTime = Boolean.parseBoolean(cl.getArgs()[2]);
-        shellState.getAccumuloClient().tableOperations().importDirectory(tableName, dir, failureDir,
-            setTime);
-        break;
-      }
       default: {
-        shellState.printException(
-            new IllegalArgumentException(String.format("Expected 2 or 3 arguments. There %s %d.",
-                args.length == 1 ? "was" : "were", args.length)));
+        shellState.printException(new IllegalArgumentException(String.format(
+            "Expected 2 arguments. There %s %d.", args.length == 1 ? "was" : "were", args.length)));
         printHelp(shellState);
         status = 1;
         break;
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
index 088fafa..5b183b8 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/InsertCommand.java
@@ -38,7 +38,7 @@
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
+import org.apache.accumulo.core.tabletingest.thrift.ConstraintViolationException;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/InterpreterCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/InterpreterCommand.java
deleted file mode 100644
index 57a184a..0000000
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/InterpreterCommand.java
+++ /dev/null
@@ -1,45 +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.shell.commands;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.interpret.ScanInterpreter;
-import org.apache.accumulo.shell.Shell;
-
-/**
- * @deprecated since 2.1.0 This will be removed in a future version
- */
-@Deprecated(since = "2.1.0")
-public class InterpreterCommand extends ShellPluginConfigurationCommand {
-
-  public InterpreterCommand() {
-    super("interpreter", Property.TABLE_INTERPRETER_CLASS, "i");
-  }
-
-  @Override
-  public String description() {
-    return "specifies a scan interpreter to interpret scan range and column arguments";
-  }
-
-  public static Class<? extends ScanInterpreter> getCurrentInterpreter(final String tableName,
-      final Shell shellState) {
-    return ShellPluginConfigurationCommand.getPluginClass(tableName, shellState,
-        ScanInterpreter.class, Property.TABLE_INTERPRETER_CLASS);
-  }
-}
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/MaxRowCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/MaxRowCommand.java
index 32eab84..044957d 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/MaxRowCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/MaxRowCommand.java
@@ -35,11 +35,7 @@
       throws Exception {
     final String tableName = OptUtil.getTableOpt(cl, shellState);
 
-    @SuppressWarnings("deprecation")
-    final org.apache.accumulo.core.util.interpret.ScanInterpreter interpeter =
-        getInterpreter(cl, tableName, shellState);
-
-    final Range range = getRange(cl, interpeter);
+    final Range range = getRange(cl);
     final Authorizations auths = getAuths(cl, shellState);
     final Text startRow = range.getStartKey() == null ? null : range.getStartKey().getRow();
     final Text endRow = range.getEndKey() == null ? null : range.getEndKey().getRow();
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/OfflineCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/OfflineCommand.java
index 0cc1d9a..36e8d5c 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/OfflineCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/OfflineCommand.java
@@ -21,7 +21,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.shell.Shell;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -40,8 +40,8 @@
   @Override
   protected void doTableOp(final Shell shellState, final String tableName)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (tableName.equals(MetadataTable.NAME)) {
-      Shell.log.info("  You cannot take the {} offline.", MetadataTable.NAME);
+    if (tableName.equals(AccumuloTable.METADATA.tableName())) {
+      Shell.log.info("  You cannot take the {} offline.", AccumuloTable.METADATA.tableName());
     } else {
       shellState.getAccumuloClient().tableOperations().offline(tableName, wait);
       Shell.log.info("Offline of table {} {}", tableName, wait ? " completed." : " initiated...");
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/OnlineCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/OnlineCommand.java
index 3426330..efa8133 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/OnlineCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/OnlineCommand.java
@@ -21,7 +21,7 @@
 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.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.shell.Shell;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -40,8 +40,8 @@
   @Override
   protected void doTableOp(final Shell shellState, final String tableName)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    if (tableName.equals(RootTable.NAME)) {
-      Shell.log.info("  The {} is always online.", RootTable.NAME);
+    if (tableName.equals(AccumuloTable.ROOT.tableName())) {
+      Shell.log.info("  The {} is always online.", AccumuloTable.ROOT.tableName());
     } else {
       shellState.getAccumuloClient().tableOperations().online(tableName, wait);
       Shell.log.info("Online of table {} {}", tableName, wait ? " completed." : " initiated...");
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
index d9bc9f7..33aa541 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java
@@ -26,7 +26,6 @@
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.classloader.ClassLoaderUtil;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
@@ -57,8 +56,8 @@
 
 public class ScanCommand extends Command {
 
-  private Option scanOptAuths, scanOptRow, scanOptColumns, disablePaginationOpt, formatterOpt,
-      interpreterOpt, formatterInterpeterOpt, outputFileOpt, scanOptCf, scanOptCq;
+  private Option scanOptAuths, scanOptRow, scanOptColumns, disablePaginationOpt, outputFileOpt,
+      scanOptCf, scanOptCq;
 
   protected Option showFewOpt;
   protected Option timestampOpt;
@@ -103,9 +102,6 @@
       final String tableName = OptUtil.getTableOpt(cl, shellState);
 
       final Class<? extends Formatter> formatter = getFormatter(cl, tableName, shellState);
-      @SuppressWarnings("deprecation")
-      final org.apache.accumulo.core.util.interpret.ScanInterpreter interpeter =
-          getInterpreter(cl, tableName, shellState);
 
       String classLoaderContext = null;
       if (cl.hasOption(contextOpt.getOpt())) {
@@ -121,11 +117,11 @@
       addScanIterators(shellState, cl, scanner, tableName);
 
       // handle remaining optional arguments
-      scanner.setRange(getRange(cl, interpeter));
+      scanner.setRange(getRange(cl));
 
       // handle columns
-      fetchColumns(cl, scanner, interpeter);
-      fetchColumsWithCFAndCQ(cl, scanner, interpeter);
+      fetchColumns(cl, scanner);
+      fetchColumsWithCFAndCQ(cl, scanner);
 
       // set timeout
       scanner.setTimeout(getTimeout(cl), TimeUnit.MILLISECONDS);
@@ -232,65 +228,12 @@
     }
   }
 
-  @Deprecated(since = "2.1.0")
-  protected org.apache.accumulo.core.util.interpret.ScanInterpreter getInterpreter(
-      final CommandLine cl, final String tableName, final Shell shellState) throws Exception {
-
-    Class<? extends org.apache.accumulo.core.util.interpret.ScanInterpreter> clazz = null;
-    try {
-      if (cl.hasOption(interpreterOpt.getOpt())) {
-        Shell.log
-            .warn("Scan Interpreter option is deprecated and will be removed in a future version.");
-
-        clazz = ClassLoaderUtil.loadClass(cl.getOptionValue(interpreterOpt.getOpt()),
-            org.apache.accumulo.core.util.interpret.ScanInterpreter.class);
-      } else if (cl.hasOption(formatterInterpeterOpt.getOpt())) {
-        Shell.log
-            .warn("Scan Interpreter option is deprecated and will be removed in a future version.");
-
-        clazz = ClassLoaderUtil.loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt()),
-            org.apache.accumulo.core.util.interpret.ScanInterpreter.class);
-      }
-    } catch (ClassNotFoundException e) {
-      Shell.log.error("Interpreter class could not be loaded.", e);
-    }
-
-    if (clazz == null) {
-      clazz = InterpreterCommand.getCurrentInterpreter(tableName, shellState);
-    }
-
-    if (clazz == null) {
-      clazz = org.apache.accumulo.core.util.interpret.DefaultScanInterpreter.class;
-    }
-
-    return clazz.getDeclaredConstructor().newInstance();
-  }
-
   protected Class<? extends Formatter> getFormatter(final CommandLine cl, final String tableName,
       final Shell shellState) throws IOException {
-
-    try {
-      if (cl.hasOption(formatterOpt.getOpt())) {
-        Shell.log.warn("Formatter option is deprecated and will be removed in a future version.");
-
-        return shellState.getClassLoader(cl, shellState)
-            .loadClass(cl.getOptionValue(formatterOpt.getOpt())).asSubclass(Formatter.class);
-      } else if (cl.hasOption(formatterInterpeterOpt.getOpt())) {
-        Shell.log.warn("Formatter option is deprecated and will be removed in a future version.");
-
-        return shellState.getClassLoader(cl, shellState)
-            .loadClass(cl.getOptionValue(formatterInterpeterOpt.getOpt()))
-            .asSubclass(Formatter.class);
-      }
-    } catch (Exception e) {
-      Shell.log.error("Formatter class could not be loaded.", e);
-    }
-
     return shellState.getFormatter(tableName);
   }
 
-  protected void fetchColumns(final CommandLine cl, final ScannerBase scanner,
-      @SuppressWarnings("deprecation") final org.apache.accumulo.core.util.interpret.ScanInterpreter formatter)
+  protected void fetchColumns(final CommandLine cl, final ScannerBase scanner)
       throws UnsupportedEncodingException {
 
     if ((cl.hasOption(scanOptCf.getOpt()) || cl.hasOption(scanOptCq.getOpt()))
@@ -306,24 +249,16 @@
       for (String a : cl.getOptionValue(scanOptColumns.getOpt()).split(",")) {
         final String[] sa = a.split(":", 2);
         if (sa.length == 1) {
-          @SuppressWarnings("deprecation")
-          var interprettedCF = formatter.interpretColumnFamily(new Text(a.getBytes(Shell.CHARSET)));
-          scanner.fetchColumnFamily(interprettedCF);
+          scanner.fetchColumnFamily(new Text(a.getBytes(Shell.CHARSET)));
         } else {
-          @SuppressWarnings("deprecation")
-          var interprettedCF =
-              formatter.interpretColumnFamily(new Text(sa[0].getBytes(Shell.CHARSET)));
-          @SuppressWarnings("deprecation")
-          var interprettedCQ =
-              formatter.interpretColumnQualifier(new Text(sa[1].getBytes(Shell.CHARSET)));
-          scanner.fetchColumn(interprettedCF, interprettedCQ);
+          scanner.fetchColumn(new Text(sa[0].getBytes(Shell.CHARSET)),
+              new Text(sa[1].getBytes(Shell.CHARSET)));
         }
       }
     }
   }
 
-  private void fetchColumsWithCFAndCQ(CommandLine cl, Scanner scanner,
-      @SuppressWarnings("deprecation") org.apache.accumulo.core.util.interpret.ScanInterpreter interpeter) {
+  private void fetchColumsWithCFAndCQ(CommandLine cl, Scanner scanner) {
     String cf = "";
     String cq = "";
     if (cl.hasOption(scanOptCf.getOpt())) {
@@ -338,24 +273,16 @@
           scanOptCf.getOpt(), scanOptCq.getOpt());
       throw new IllegalArgumentException(formattedString);
     } else if (!cf.isEmpty() && cq.isEmpty()) {
-      @SuppressWarnings("deprecation")
-      var interprettedCF = interpeter.interpretColumnFamily(new Text(cf.getBytes(Shell.CHARSET)));
-      scanner.fetchColumnFamily(interprettedCF);
+      scanner.fetchColumnFamily(new Text(cf.getBytes(Shell.CHARSET)));
     } else if (!cf.isEmpty() && !cq.isEmpty()) {
-      @SuppressWarnings("deprecation")
-      var interprettedCF = interpeter.interpretColumnFamily(new Text(cf.getBytes(Shell.CHARSET)));
-      @SuppressWarnings("deprecation")
-      var interprettedCQ =
-          interpeter.interpretColumnQualifier(new Text(cq.getBytes(Shell.CHARSET)));
-      scanner.fetchColumn(interprettedCF, interprettedCQ);
+      scanner.fetchColumn(new Text(cf.getBytes(Shell.CHARSET)),
+          new Text(cq.getBytes(Shell.CHARSET)));
 
     }
 
   }
 
-  protected Range getRange(final CommandLine cl,
-      @SuppressWarnings("deprecation") final org.apache.accumulo.core.util.interpret.ScanInterpreter formatter)
-      throws UnsupportedEncodingException {
+  protected Range getRange(final CommandLine cl) throws UnsupportedEncodingException {
     if ((cl.hasOption(OptUtil.START_ROW_OPT) || cl.hasOption(OptUtil.END_ROW_OPT))
         && cl.hasOption(scanOptRow.getOpt())) {
       // did not see a way to make commons cli do this check... it has mutually exclusive options
@@ -365,23 +292,10 @@
     }
 
     if (cl.hasOption(scanOptRow.getOpt())) {
-      @SuppressWarnings("deprecation")
-      var interprettedRow = formatter
-          .interpretRow(new Text(cl.getOptionValue(scanOptRow.getOpt()).getBytes(Shell.CHARSET)));
-      return new Range(interprettedRow);
+      return new Range(new Text(cl.getOptionValue(scanOptRow.getOpt()).getBytes(Shell.CHARSET)));
     } else {
       Text startRow = OptUtil.getStartRow(cl);
-      if (startRow != null) {
-        @SuppressWarnings("deprecation")
-        var interprettedBeginRow = formatter.interpretBeginRow(startRow);
-        startRow = interprettedBeginRow;
-      }
       Text endRow = OptUtil.getEndRow(cl);
-      if (endRow != null) {
-        @SuppressWarnings("deprecation")
-        var interprettedEndRow = formatter.interpretEndRow(endRow);
-        endRow = interprettedEndRow;
-      }
       final boolean startInclusive = !cl.hasOption(optStartRowExclusive.getOpt());
       final boolean endInclusive = !cl.hasOption(optEndRowExclusive.getOpt());
       return new Range(startRow, startInclusive, endRow, endInclusive);
@@ -432,12 +346,6 @@
     timestampOpt = new Option("st", "show-timestamps", false, "display timestamps");
     disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
     showFewOpt = new Option("f", "show-few", true, "show only a specified number of characters");
-    formatterOpt =
-        new Option("fm", "formatter", true, "fully qualified name of the formatter class to use");
-    interpreterOpt = new Option("i", "interpreter", true,
-        "fully qualified name of the interpreter class to use");
-    formatterInterpeterOpt = new Option("fi", "fmt-interpreter", true,
-        "fully qualified name of a class that is a formatter and interpreter");
     timeoutOption = new Option(null, "timeout", true,
         "time before scan should fail if no data is returned. If no unit is"
             + " given assumes seconds. Units d,h,m,s,and ms are supported. e.g. 30s or 100ms");
@@ -456,7 +364,6 @@
     scanOptCq.setArgName("column-qualifier");
     showFewOpt.setRequired(false);
     showFewOpt.setArgName("int");
-    formatterOpt.setArgName("className");
     timeoutOption.setArgName("timeout");
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
@@ -481,9 +388,6 @@
     o.addOption(timestampOpt);
     o.addOption(disablePaginationOpt);
     o.addOption(OptUtil.tableOpt("table to be scanned"));
-    o.addOption(formatterOpt);
-    o.addOption(interpreterOpt);
-    o.addOption(formatterInterpeterOpt);
     o.addOption(timeoutOption);
     if (Arrays.asList(ScanCommand.class.getName(), GrepCommand.class.getName(),
         EGrepCommand.class.getName()).contains(this.getClass().getName())) {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java
deleted file mode 100644
index b5350d6..0000000
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/ScriptCommand.java
+++ /dev/null
@@ -1,317 +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.shell.commands;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.Reader;
-import java.io.Writer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import javax.script.Bindings;
-import javax.script.Compilable;
-import javax.script.CompiledScript;
-import javax.script.Invocable;
-import javax.script.ScriptContext;
-import javax.script.ScriptEngine;
-import javax.script.ScriptEngineFactory;
-import javax.script.ScriptEngineManager;
-import javax.script.ScriptException;
-import javax.script.SimpleScriptContext;
-
-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.shell.Shell;
-import org.apache.accumulo.shell.Shell.Command;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionGroup;
-import org.apache.commons.cli.Options;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * @deprecated since 2.0; this command shouldn't be used; The script command is deprecated; use
- *             jshell for scripting instead
- */
-
-@Deprecated(since = "2.1.0")
-public class ScriptCommand extends Command {
-
-  // Command to allow user to run scripts, see JSR-223
-  // https://www.oracle.com/technetwork/articles/javase/scripting-140262.html
-
-  protected Option list, engine, script, file, args, out, function, object;
-  private static final String DEFAULT_ENGINE = "rhino";
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "app is run in same security context as user providing the filename")
-  @Override
-  public int execute(String fullCommand, CommandLine cl, Shell shellState) throws Exception {
-
-    boolean invoke = false;
-
-    Shell.log.warn("The script command is deprecated; use jshell for scripting instead");
-    ScriptEngineManager mgr = new ScriptEngineManager();
-
-    if (cl.hasOption(list.getOpt())) {
-      listJSREngineInfo(mgr, shellState);
-    } else if (cl.hasOption(file.getOpt()) || cl.hasOption(script.getOpt())) {
-      String engineName = DEFAULT_ENGINE;
-      if (cl.hasOption(engine.getOpt())) {
-        engineName = cl.getOptionValue(engine.getOpt());
-      }
-      ScriptEngine engine = mgr.getEngineByName(engineName);
-      if (engine == null) {
-        shellState.printException(new Exception(engineName + " not found"));
-        return 1;
-      }
-
-      if (cl.hasOption(object.getOpt()) || cl.hasOption(function.getOpt())) {
-        if (!(engine instanceof Invocable)) {
-          shellState.printException(
-              new Exception(engineName + " does not support invoking functions or methods"));
-          return 1;
-        }
-        invoke = true;
-      }
-
-      ScriptContext ctx = new SimpleScriptContext();
-
-      // Put the following objects into the context so that they
-      // are available to the scripts
-      // TODO: What else should go in here?
-      Bindings b = engine.getBindings(ScriptContext.ENGINE_SCOPE);
-      putConnector(b, shellState.getAccumuloClient());
-      b.put("client", shellState.getAccumuloClient());
-
-      List<Object> argValues = new ArrayList<>();
-      if (cl.hasOption(args.getOpt())) {
-        String[] argList = cl.getOptionValue(args.getOpt()).split(",");
-        for (String arg : argList) {
-          String[] parts = arg.split("=");
-          if (parts.length == 0) {
-            continue;
-          } else if (parts.length == 1) {
-            b.put(parts[0], null);
-            argValues.add(null);
-          } else if (parts.length == 2) {
-            b.put(parts[0], parts[1]);
-            argValues.add(parts[1]);
-          }
-        }
-      }
-      ctx.setBindings(b, ScriptContext.ENGINE_SCOPE);
-      Object[] argArray = argValues.toArray(new Object[argValues.size()]);
-
-      Writer writer = null;
-      if (cl.hasOption(out.getOpt())) {
-        File f = new File(cl.getOptionValue(out.getOpt()));
-        writer = new FileWriter(f, UTF_8);
-        ctx.setWriter(writer);
-      }
-
-      if (cl.hasOption(file.getOpt())) {
-        File f = new File(cl.getOptionValue(file.getOpt()));
-        if (!f.exists()) {
-          if (writer != null) {
-            writer.close();
-          }
-          shellState.printException(new Exception(f.getAbsolutePath() + " not found"));
-          return 1;
-        }
-        Reader reader = new FileReader(f, UTF_8);
-        try (reader) {
-          engine.eval(reader, ctx);
-          if (invoke) {
-            this.invokeFunctionOrMethod(shellState, engine, cl, argArray);
-          }
-        } catch (ScriptException ex) {
-          shellState.printException(ex);
-          return 1;
-        } finally {
-          if (writer != null) {
-            writer.close();
-          }
-        }
-      } else if (cl.hasOption(script.getOpt())) {
-        String inlineScript = cl.getOptionValue(script.getOpt());
-        try {
-          if (engine instanceof Compilable) {
-            Compilable compiledEng = (Compilable) engine;
-            CompiledScript script = compiledEng.compile(inlineScript);
-            script.eval(ctx);
-            if (invoke) {
-              this.invokeFunctionOrMethod(shellState, engine, cl, argArray);
-            }
-          } else {
-            engine.eval(inlineScript, ctx);
-            if (invoke) {
-              this.invokeFunctionOrMethod(shellState, engine, cl, argArray);
-            }
-          }
-        } catch (ScriptException ex) {
-          shellState.printException(ex);
-          return 1;
-        } finally {
-          if (writer != null) {
-            writer.close();
-          }
-        }
-      }
-      if (writer != null) {
-        writer.close();
-      }
-
-    } else {
-      printHelp(shellState);
-    }
-    return 0;
-  }
-
-  private void putConnector(Bindings b, AccumuloClient client) {
-    try {
-      b.put("connection", org.apache.accumulo.core.client.Connector.from(client));
-    } catch (AccumuloSecurityException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public String description() {
-    return "(deprecated) execute JSR-223 scripts";
-  }
-
-  @Override
-  public int numArgs() {
-    return 0;
-  }
-
-  @Override
-  public Options getOptions() {
-    final Options o = new Options();
-
-    engine = new Option("e", "engine", false, "engine name, defaults to JDK default (Rhino)");
-    engine.setArgName("engineName");
-    engine.setArgs(1);
-    engine.setRequired(false);
-    o.addOption(engine);
-
-    OptionGroup inputGroup = new OptionGroup();
-    list = new Option("l", "list", false, "list available script engines");
-    inputGroup.addOption(list);
-
-    script = new Option("s", "script", true, "use inline script");
-    script.setArgName("script text");
-    script.setArgs(1);
-    script.setRequired(false);
-    inputGroup.addOption(script);
-
-    file = new Option("f", "file", true, "use script file");
-    file.setArgName("fileName");
-    file.setArgs(1);
-    file.setRequired(false);
-
-    inputGroup.addOption(file);
-    inputGroup.setRequired(true);
-    o.addOptionGroup(inputGroup);
-
-    OptionGroup invokeGroup = new OptionGroup();
-    object = new Option("obj", "object", true, "name of object");
-    object.setArgs(1);
-    object.setArgName("objectName:methodName");
-    object.setRequired(false);
-    invokeGroup.addOption(object);
-
-    function = new Option("fx", "function", true, "invoke a script function");
-    function.setArgName("functionName");
-    function.setArgs(1);
-    function.setRequired(false);
-    invokeGroup.addOption(function);
-    invokeGroup.setRequired(false);
-    o.addOptionGroup(invokeGroup);
-
-    args = new Option("a", "args", true, "comma separated list of key=value arguments");
-    args.setArgName("property1=value1,propert2=value2,...");
-    args.setArgs(Option.UNLIMITED_VALUES);
-    args.setRequired(false);
-    o.addOption(args);
-
-    out = new Option("o", "output", true, "output file");
-    out.setArgName("fileName");
-    out.setArgs(1);
-    out.setRequired(false);
-    o.addOption(out);
-
-    return o;
-  }
-
-  private void listJSREngineInfo(ScriptEngineManager mgr, Shell shellState) throws IOException {
-    List<ScriptEngineFactory> factories = mgr.getEngineFactories();
-    Set<String> lines = new TreeSet<>();
-    for (ScriptEngineFactory factory : factories) {
-      lines.add("ScriptEngineFactory Info");
-      String engName = factory.getEngineName();
-      String engVersion = factory.getEngineVersion();
-      String langName = factory.getLanguageName();
-      String langVersion = factory.getLanguageVersion();
-      lines.add("\tScript Engine: " + engName + " (" + engVersion + ")");
-      List<String> engNames = factory.getNames();
-      for (String name : engNames) {
-        lines.add("\tEngine Alias: " + name);
-      }
-      lines.add("\tLanguage: " + langName + " (" + langVersion + ")");
-    }
-    shellState.printLines(lines.iterator(), true);
-
-  }
-
-  private void invokeFunctionOrMethod(Shell shellState, ScriptEngine engine, CommandLine cl,
-      Object[] args) {
-    try {
-      Invocable inv = (Invocable) engine;
-      if (cl.hasOption(function.getOpt())) {
-        inv.invokeFunction(cl.getOptionValue(function.getOpt()), args);
-      } else if (cl.hasOption(object.getOpt())) {
-        String objectMethod = cl.getOptionValue(object.getOpt());
-        String[] parts = objectMethod.split(":");
-        if (parts.length != 2) {
-          shellState.printException(new Exception("Object and Method must be supplied"));
-          return;
-        }
-        String objectName = parts[0];
-        String methodName = parts[1];
-        Object obj = engine.get(objectName);
-        inv.invokeMethod(obj, methodName, args);
-
-      }
-    } catch (Exception e) {
-      shellState.printException(e);
-    }
-  }
-
-}
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
index 48e8bef..a370bed 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/SetIterCommand.java
@@ -40,7 +40,7 @@
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.ReqVisFilter;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.Shell.Command;
 import org.apache.accumulo.shell.ShellCommandException;
@@ -92,7 +92,7 @@
     String configuredName;
     try {
       if (profileOpt != null && (currentTableName == null || currentTableName.isBlank())) {
-        tmpTable = MetadataTable.NAME;
+        tmpTable = AccumuloTable.METADATA.tableName();
         shellState.setTableName(tmpTable);
         tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !currentTableName.isEmpty();
       }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/SetScanIterCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/SetScanIterCommand.java
deleted file mode 100644
index 2cb2295..0000000
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/SetScanIterCommand.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.shell.commands;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.shell.Shell;
-import org.apache.accumulo.shell.ShellCommandException;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionGroup;
-import org.apache.commons.cli.Options;
-
-@Deprecated
-public class SetScanIterCommand extends SetIterCommand {
-  @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, IOException,
-      ShellCommandException {
-    Shell.log.warn("Deprecated, use {}", new SetShellIterCommand().getName());
-    return super.execute(fullCommand, cl, shellState);
-  }
-
-  @Override
-  protected void setTableProperties(final CommandLine cl, final Shell shellState,
-      final int priority, final Map<String,String> options, final String classname,
-      final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException,
-      TableNotFoundException {
-
-    final String tableName = OptUtil.getTableOpt(cl, shellState);
-
-    ScanCommand.ensureTserversCanLoadIterator(shellState, tableName, classname);
-
-    options.values().removeIf(v -> v == null || v.isEmpty());
-
-    List<IteratorSetting> tableScanIterators =
-        shellState.scanIteratorOptions.computeIfAbsent(tableName, k -> new ArrayList<>());
-    final IteratorSetting setting = new IteratorSetting(priority, name, classname);
-    setting.addOptions(options);
-
-    // initialize a scanner to ensure the new setting does not conflict with existing settings
-    final String user = shellState.getAccumuloClient().whoami();
-    final Authorizations auths =
-        shellState.getAccumuloClient().securityOperations().getUserAuthorizations(user);
-    final Scanner scanner = shellState.getAccumuloClient().createScanner(tableName, auths);
-    for (IteratorSetting s : tableScanIterators) {
-      scanner.addScanIterator(s);
-    }
-    scanner.addScanIterator(setting);
-
-    // if no exception has been thrown, it's safe to add it to the list
-    tableScanIterators.add(setting);
-    Shell.log.debug("Scan iterators :{}", shellState.scanIteratorOptions.get(tableName));
-  }
-
-  @Override
-  public String description() {
-    return "(deprecated) sets a table-specific scan iterator for this shell session";
-  }
-
-  @Override
-  public Options getOptions() {
-    // Remove the options that specify which type of iterator this is, since
-    // they are all scan iterators with this command.
-    final HashSet<OptionGroup> groups = new HashSet<>();
-    final Options parentOptions = super.getOptions();
-    final Options modifiedOptions = new Options();
-    for (Option o : parentOptions.getOptions()) {
-      if (!IteratorScope.majc.name().equals(o.getOpt())
-          && !IteratorScope.minc.name().equals(o.getOpt())
-          && !IteratorScope.scan.name().equals(o.getOpt())) {
-        modifiedOptions.addOption(o);
-        OptionGroup group = parentOptions.getOptionGroup(o);
-        if (group != null) {
-          groups.add(group);
-        }
-      }
-    }
-    for (OptionGroup group : groups) {
-      modifiedOptions.addOptionGroup(group);
-    }
-    return modifiedOptions;
-  }
-
-}
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/DeleteTableCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/DeleteTableCommandTest.java
index f935625..3bcdd56 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/DeleteTableCommandTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/commands/DeleteTableCommandTest.java
@@ -22,8 +22,7 @@
 
 import java.util.Set;
 
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Sets;
@@ -32,7 +31,8 @@
 
   @Test
   public void removeAccumuloNamespaceTables() {
-    Set<String> tables = Sets.newHashSet(MetadataTable.NAME, RootTable.NAME, "a1", "a2");
+    Set<String> tables = Sets.newHashSet(AccumuloTable.METADATA.tableName(),
+        AccumuloTable.ROOT.tableName(), "a1", "a2");
     DeleteTableCommand cmd = new DeleteTableCommand();
     cmd.pruneTables(tables);
 
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java
deleted file mode 100644
index 856c7cb..0000000
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/FateCommandTest.java
+++ /dev/null
@@ -1,349 +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.shell.commands;
-
-import static org.apache.accumulo.core.Constants.ZTABLE_LOCKS;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.reset;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.FileDescriptor;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.PrintWriter;
-import java.nio.file.Files;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.fate.AdminUtil;
-import org.apache.accumulo.core.fate.ReadOnlyRepo;
-import org.apache.accumulo.core.fate.ReadOnlyTStore;
-import org.apache.accumulo.core.fate.ZooStore;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.shell.Shell;
-import org.apache.accumulo.shell.ShellConfigTest.TestOutputStream;
-import org.apache.commons.cli.CommandLine;
-import org.apache.zookeeper.KeeperException;
-import org.jline.reader.LineReader;
-import org.jline.reader.LineReaderBuilder;
-import org.jline.terminal.Size;
-import org.jline.terminal.Terminal;
-import org.jline.terminal.impl.DumbTerminal;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-public class FateCommandTest {
-
-  public static class TestFateCommand extends FateCommand {
-
-    private boolean dumpCalled = false;
-    private boolean deleteCalled = false;
-    private boolean failCalled = false;
-    private boolean printCalled = false;
-
-    @Override
-    public String getName() {
-      return "fate";
-    }
-
-    @Override
-    protected String getZKRoot(ClientContext context) {
-      return "";
-    }
-
-    @Override
-    synchronized ZooReaderWriter getZooReaderWriter(ClientContext context, String secret) {
-      return null;
-    }
-
-    @Override
-    protected ZooStore<FateCommand> getZooStore(String fateZkPath, ZooReaderWriter zrw)
-        throws KeeperException, InterruptedException {
-      return null;
-    }
-
-    @Override
-    String dumpTx(ZooStore<FateCommand> zs, String[] args) {
-      dumpCalled = true;
-      return "";
-    }
-
-    @Override
-    protected boolean deleteTx(PrintWriter out, AdminUtil<FateCommand> admin,
-        ZooStore<FateCommand> zs, ZooReaderWriter zk, ServiceLockPath zLockManagerPath,
-        String[] args) throws InterruptedException, KeeperException {
-      deleteCalled = true;
-      return true;
-    }
-
-    @Override
-    public boolean failTx(PrintWriter out, AdminUtil<FateCommand> admin, ZooStore<FateCommand> zs,
-        ZooReaderWriter zk, ServiceLockPath managerLockPath, String[] args) {
-      failCalled = true;
-      return true;
-    }
-
-    @Override
-    protected void printTx(Shell shellState, AdminUtil<FateCommand> admin, ZooStore<FateCommand> zs,
-        ZooReaderWriter zk, ServiceLockPath tableLocksPath, String[] args, CommandLine cl)
-        throws InterruptedException, KeeperException, IOException {
-      printCalled = true;
-    }
-
-    public void reset() {
-      dumpCalled = false;
-      deleteCalled = false;
-      failCalled = false;
-      printCalled = false;
-    }
-
-  }
-
-  private static ZooReaderWriter zk;
-  private static ServiceLockPath managerLockPath;
-
-  @BeforeAll
-  public static void setup() {
-    zk = createMock(ZooReaderWriter.class);
-    managerLockPath = createMock(ServiceLockPath.class);
-  }
-
-  @Test
-  public void testFailTx() throws Exception {
-    ZooStore<FateCommand> zs = createMock(ZooStore.class);
-    String tidStr = "12345";
-    long tid = Long.parseLong(tidStr, 16);
-    expect(zs.getStatus(tid)).andReturn(ReadOnlyTStore.TStatus.NEW).anyTimes();
-    zs.reserve(tid);
-    expectLastCall().once();
-    zs.setStatus(tid, ReadOnlyTStore.TStatus.FAILED_IN_PROGRESS);
-    expectLastCall().once();
-    zs.unreserve(tid, 0, TimeUnit.MILLISECONDS);
-    expectLastCall().once();
-
-    TestHelper helper = new TestHelper(true);
-
-    replay(zs);
-
-    FateCommand cmd = new FateCommand();
-    // require number for Tx
-    var out = new PrintWriter(System.out);
-    assertFalse(cmd.failTx(out, helper, zs, zk, managerLockPath, new String[] {"fail", "tx1"}));
-    // fail the long configured above
-    assertTrue(cmd.failTx(out, helper, zs, zk, managerLockPath, new String[] {"fail", "12345"}));
-
-    verify(zs);
-  }
-
-  @Test
-  public void testDump() {
-    ZooStore<FateCommand> zs = createMock(ZooStore.class);
-    ReadOnlyRepo<FateCommand> ser = createMock(ReadOnlyRepo.class);
-    long tid1 = Long.parseLong("12345", 16);
-    long tid2 = Long.parseLong("23456", 16);
-    expect(zs.getStack(tid1)).andReturn(List.of(ser)).once();
-    expect(zs.getStack(tid2)).andReturn(List.of(ser)).once();
-
-    replay(zs);
-
-    FateCommand cmd = new FateCommand();
-
-    var args = new String[] {"dump", "12345", "23456"};
-    var output = cmd.dumpTx(zs, args);
-    System.out.println(output);
-    assertTrue(output.contains("0000000000012345"));
-    assertTrue(output.contains("0000000000023456"));
-
-    verify(zs);
-  }
-
-  @Test
-  public void testPrintAndList() throws IOException, InterruptedException, KeeperException {
-    reset(zk);
-    PrintStream out = System.out;
-    File config = Files.createTempFile(null, null).toFile();
-    TestOutputStream output = new TestOutputStream();
-    Shell shell = createShell(output);
-
-    ServiceLockPath tableLocksPath = ServiceLock.path("/accumulo" + ZTABLE_LOCKS);
-    ZooStore<FateCommand> zs = createMock(ZooStore.class);
-    expect(zk.getChildren(tableLocksPath.toString())).andReturn(List.of("5")).anyTimes();
-    expect(zk.getChildren("/accumulo/table_locks/5")).andReturn(List.of()).anyTimes();
-    expect(zs.list()).andReturn(List.of()).anyTimes();
-
-    replay(zs, zk);
-
-    TestHelper helper = new TestHelper(true);
-    FateCommand cmd = new FateCommand();
-    var options = cmd.getOptions();
-    CommandLine cli = new CommandLine.Builder().addOption(options.getOption("list"))
-        .addOption(options.getOption("print")).addOption(options.getOption("np")).build();
-
-    try {
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list"), cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list FATE[1]"), cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list 1234"), cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("list 1234 2345"),
-          cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print"), cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print FATE[1]"), cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print 1234"), cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, cli.getOptionValues("print 1234 2345"),
-          cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {""}, cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, new String[] {}, cli);
-      cmd.printTx(shell, helper, zs, zk, tableLocksPath, null, cli);
-    } finally {
-      output.clear();
-      System.setOut(out);
-      if (config.exists()) {
-        assertTrue(config.delete());
-      }
-    }
-
-    verify(zs, zk);
-  }
-
-  @Test
-  public void testCommandLineOptions() throws Exception {
-    PrintStream out = System.out;
-    File config = Files.createTempFile(null, null).toFile();
-    TestOutputStream output = new TestOutputStream();
-
-    Shell shell = createShell(output);
-    shell.setLogErrorsToConsole();
-    try {
-      assertTrue(shell.config("--config-file", config.toString(), "-zh", "127.0.0.1:2181", "-zi",
-          "test", "-u", "test", "-p", "password"));
-      TestFateCommand cmd = new TestFateCommand();
-      shell.commandFactory.clear();
-      shell.commandFactory.put("fate", cmd);
-      shell.execCommand("fate -?", true, false);
-      Shell.log.info("{}", output.get());
-      shell.execCommand("fate --help", true, false);
-      cmd.reset();
-      shell.execCommand("fate delete", true, false);
-      assertFalse(cmd.deleteCalled);
-      cmd.reset();
-      shell.execCommand("fate -delete", true, false);
-      assertFalse(cmd.deleteCalled);
-      cmd.reset();
-      shell.execCommand("fate -delete 12345", true, false);
-      assertTrue(cmd.deleteCalled);
-      cmd.reset();
-      shell.execCommand("fate --delete 12345 67890", true, false);
-      assertTrue(cmd.deleteCalled);
-      cmd.reset();
-      shell.execCommand("fate dump", true, false);
-      assertFalse(cmd.dumpCalled);
-      cmd.reset();
-      shell.execCommand("fate -dump", true, false);
-      assertTrue(cmd.dumpCalled);
-      cmd.reset();
-      shell.execCommand("fate -dump 12345", true, false);
-      assertTrue(cmd.dumpCalled);
-      cmd.reset();
-      shell.execCommand("fate --dump 12345 67890", true, false);
-      assertTrue(cmd.dumpCalled);
-      cmd.reset();
-      shell.execCommand("fate fail", true, false);
-      assertFalse(cmd.failCalled);
-      cmd.reset();
-      shell.execCommand("fate -fail", true, false);
-      assertFalse(cmd.failCalled);
-      cmd.reset();
-      shell.execCommand("fate -fail 12345", true, false);
-      assertTrue(cmd.failCalled);
-      cmd.reset();
-      shell.execCommand("fate --fail 12345 67890", true, false);
-      assertTrue(cmd.failCalled);
-      cmd.reset();
-      shell.execCommand("fate print", true, false);
-      assertFalse(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate -print", true, false);
-      assertTrue(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate --print", true, false);
-      assertTrue(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate --print 12345 67890", true, false);
-      assertTrue(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate list", true, false);
-      assertFalse(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate -list", true, false);
-      assertTrue(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate --list", true, false);
-      assertTrue(cmd.printCalled);
-      cmd.reset();
-      shell.execCommand("fate --list 12345 67890", true, false);
-      assertTrue(cmd.printCalled);
-      cmd.reset();
-    } finally {
-      shell.shutdown();
-      output.clear();
-      System.setOut(out);
-      if (config.exists()) {
-        assertTrue(config.delete());
-      }
-    }
-  }
-
-  private Shell createShell(TestOutputStream output) throws IOException {
-    System.setOut(new PrintStream(output));
-    Terminal terminal = new DumbTerminal(new FileInputStream(FileDescriptor.in), output);
-    terminal.setSize(new Size(80, 24));
-    LineReader reader = LineReaderBuilder.builder().terminal(terminal).build();
-    Shell shell = new Shell(reader) {
-      @Override
-      protected boolean authenticateUser(AccumuloClient client, AuthenticationToken token) {
-        return true;
-      }
-    };
-    shell.setLogErrorsToConsole();
-    return shell;
-  }
-
-  static class TestHelper extends AdminUtil<FateCommand> {
-
-    public TestHelper(boolean exitOnError) {
-      super(exitOnError);
-    }
-
-    @Override
-    public boolean checkGlobalLock(ZooReaderWriter zk, ServiceLockPath zLockManagerPath) {
-      return true;
-    }
-  }
-}
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/ListBulkCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/ListBulkCommandTest.java
index d62ad4d..6434937 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/ListBulkCommandTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/commands/ListBulkCommandTest.java
@@ -29,10 +29,10 @@
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.accumulo.core.manager.thrift.BulkImportState;
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.BulkImportState;
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.junit.jupiter.api.Test;
 
 public class ListBulkCommandTest {
diff --git a/shell/src/test/resources/log4j2-test.properties b/shell/src/test/resources/log4j2-test.properties
index 2e574af..42b0851 100644
--- a/shell/src/test/resources/log4j2-test.properties
+++ b/shell/src/test/resources/log4j2-test.properties
@@ -36,9 +36,6 @@
 logger.03.name = org.apache.accumulo.core.iterators.user.TransformingIteratorTest$IllegalVisKeyTransformingIterator
 logger.03.level = fatal
 
-logger.04.name = org.apache.commons.vfs2.impl.DefaultFileSystemManager
-logger.04.level = warn
-
 logger.05.name = org.apache.hadoop.mapred
 logger.05.level = error
 
@@ -53,4 +50,3 @@
 
 rootLogger.level = info
 rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/src/build/ci/find-unapproved-junit.sh b/src/build/ci/find-unapproved-junit.sh
index 8007ecc..da8ac47 100755
--- a/src/build/ci/find-unapproved-junit.sh
+++ b/src/build/ci/find-unapproved-junit.sh
@@ -23,9 +23,6 @@
 # APIs to new jupiter tests.
 NUM_EXPECTED=0
 ALLOWED=(
-  server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java
-  server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java
-  start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java
 )
 
 ALLOWED_PIPE_SEP=$({ for x in "${ALLOWED[@]}"; do echo "$x"; done; } | paste -sd'|')
@@ -39,8 +36,10 @@
   fi
   # find any new classes using something other than the jupiter API, except those allowed
   grep "$opts" --include='*.java' 'org[.]junit[.](?!jupiter)' | grep -Pv "^(${ALLOWED_PIPE_SEP//./[.]})\$"
-  # find any uses of the jupiter API in the allowed vintage classes
-  grep "$opts" 'org[.]junit[.]jupiter' "${ALLOWED[@]}"
+  if ((${#ALLOWED[@]} != 0)); then
+    # find any uses of the jupiter API in the allowed vintage classes
+    grep "$opts" 'org[.]junit[.]jupiter' "${ALLOWED[@]}"
+  fi
 }
 
 function comparecounts() {
diff --git a/start/pom.xml b/start/pom.xml
index 44efb3b..9ead035 100644
--- a/start/pom.xml
+++ b/start/pom.xml
@@ -24,33 +24,13 @@
   <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-start</artifactId>
   <name>Apache Accumulo Start</name>
   <description>A library for launching Apache Accumulo services.</description>
   <dependencies>
     <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-configuration2</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-vfs2</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client-api</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
@@ -60,8 +40,8 @@
       <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client-minicluster</artifactId>
+      <groupId>com.github.spotbugs</groupId>
+      <artifactId>spotbugs-annotations</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -74,26 +54,6 @@
       <artifactId>log4j-slf4j2-impl</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-api</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.junit.vintage</groupId>
-      <artifactId>junit-vintage-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.powermock</groupId>
-      <artifactId>powermock-api-easymock</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.powermock</groupId>
-      <artifactId>powermock-module-junit4</artifactId>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
   <build>
     <pluginManagement>
@@ -116,6 +76,13 @@
             <reuseForks>false</reuseForks>
           </configuration>
         </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-failsafe-plugin</artifactId>
+          <configuration>
+            <skipITs>true</skipITs>
+          </configuration>
+        </plugin>
       </plugins>
     </pluginManagement>
     <plugins>
diff --git a/start/src/main/java/org/apache/accumulo/start/Main.java b/start/src/main/java/org/apache/accumulo/start/Main.java
index e736821..3f6eea7 100644
--- a/start/src/main/java/org/apache/accumulo/start/Main.java
+++ b/start/src/main/java/org/apache/accumulo/start/Main.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.start;
 
-import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -38,37 +37,10 @@
 
   private static final Logger log = LoggerFactory.getLogger(Main.class);
   private static ClassLoader classLoader;
-  private static Class<?> vfsClassLoader;
   private static Map<String,KeywordExecutable> servicesMap;
 
   public static void main(final String[] args) throws Exception {
-    // Preload classes that cause a deadlock between the ServiceLoader and the DFSClient when
-    // using the VFSClassLoader with jars in HDFS.
-    ClassLoader loader = getClassLoader();
-    Class<?> confClass = null;
-    try {
-      @SuppressWarnings("deprecation")
-      var deprecatedConfClass = org.apache.accumulo.start.classloader.AccumuloClassLoader
-          .getClassLoader().loadClass("org.apache.hadoop.conf.Configuration");
-      confClass = deprecatedConfClass;
-      Object conf = null;
-      try {
-        conf = confClass.getDeclaredConstructor().newInstance();
-        try {
-          Method getClassByNameOrNullMethod =
-              conf.getClass().getMethod("getClassByNameOrNull", String.class);
-          getClassByNameOrNullMethod.invoke(conf, "org.apache.hadoop.mapred.JobConf");
-          getClassByNameOrNullMethod.invoke(conf, "org.apache.hadoop.mapred.JobConfigurable");
-        } catch (Exception e) {
-          die(e, "Error pre-loading JobConf and JobConfigurable classes, VFS classloader with "
-              + "system classes in HDFS may not work correctly");
-        }
-      } catch (Exception e) {
-        die(e, "Error creating new instance of Hadoop Configuration");
-      }
-    } catch (ClassNotFoundException e) {
-      die(e, "Unable to find Hadoop Configuration class on classpath, check configuration.");
-    }
+    final ClassLoader loader = getClassLoader();
 
     if (args.length == 0) {
       printUsage();
@@ -94,28 +66,15 @@
   public static synchronized ClassLoader getClassLoader() {
     if (classLoader == null) {
       try {
-        classLoader = (ClassLoader) getVFSClassLoader().getMethod("getClassLoader").invoke(null);
+        classLoader = ClassLoader.getSystemClassLoader();
         Thread.currentThread().setContextClassLoader(classLoader);
-      } catch (IOException | IllegalArgumentException | ReflectiveOperationException
-          | SecurityException e) {
+      } catch (IllegalArgumentException | SecurityException e) {
         die(e, "Problem initializing the class loader");
       }
     }
     return classLoader;
   }
 
-  @Deprecated
-  private static synchronized Class<?> getVFSClassLoader()
-      throws IOException, ClassNotFoundException {
-    if (vfsClassLoader == null) {
-      Thread.currentThread().setContextClassLoader(
-          org.apache.accumulo.start.classloader.AccumuloClassLoader.getClassLoader());
-      vfsClassLoader = org.apache.accumulo.start.classloader.AccumuloClassLoader.getClassLoader()
-          .loadClass("org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader");
-    }
-    return vfsClassLoader;
-  }
-
   private static void execKeyword(final KeywordExecutable keywordExec, final String[] args) {
     Runnable r = () -> {
       try {
diff --git a/start/src/main/java/org/apache/accumulo/start/TestMain.java b/start/src/main/java/org/apache/accumulo/start/TestMain.java
index 6a0e238..92e87e5 100644
--- a/start/src/main/java/org/apache/accumulo/start/TestMain.java
+++ b/start/src/main/java/org/apache/accumulo/start/TestMain.java
@@ -34,7 +34,7 @@
         System.exit(0);
       }
       if (args[0].equals("throw")) {
-        throw new RuntimeException("This is an exception");
+        throw new IllegalStateException("This is an exception");
       }
     }
     System.exit(-1);
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java
deleted file mode 100644
index 3a816dc..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/AccumuloClassLoader.java
+++ /dev/null
@@ -1,224 +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.start.classloader;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.configuration2.PropertiesConfiguration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated
-public class AccumuloClassLoader {
-
-  public static final String GENERAL_CLASSPATHS = "general.classpaths";
-
-  private static URL accumuloConfigUrl;
-  private static URLClassLoader classloader;
-  private static final Logger log = LoggerFactory.getLogger(AccumuloClassLoader.class);
-
-  static {
-    String configFile = System.getProperty("accumulo.properties", "accumulo.properties");
-    if (configFile.startsWith("file://")) {
-      try {
-        File f = new File(new URI(configFile));
-        if (f.exists() && !f.isDirectory()) {
-          accumuloConfigUrl = f.toURI().toURL();
-        } else {
-          log.warn("Failed to load Accumulo configuration from " + configFile);
-        }
-      } catch (URISyntaxException | MalformedURLException e) {
-        log.warn("Failed to load Accumulo configuration from " + configFile, e);
-      }
-    } else {
-      accumuloConfigUrl = AccumuloClassLoader.class.getClassLoader().getResource(configFile);
-      if (accumuloConfigUrl == null) {
-        log.warn("Failed to load Accumulo configuration '{}' from classpath", configFile);
-      }
-    }
-    if (accumuloConfigUrl != null) {
-      log.debug("Using Accumulo configuration at {}", accumuloConfigUrl.getFile());
-    }
-  }
-
-  /**
-   * Returns value of property in accumulo.properties file, otherwise default value
-   *
-   * @param propertyName Name of the property to pull
-   * @param defaultValue Value to default to if not found.
-   * @return value of property or default
-   */
-  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
-      justification = "url is specified by an admin, not unchecked user input")
-  public static String getAccumuloProperty(String propertyName, String defaultValue) {
-    if (accumuloConfigUrl == null) {
-      log.warn(
-          "Using default value '{}' for '{}' as there is no Accumulo configuration on classpath",
-          defaultValue, propertyName);
-      return defaultValue;
-    }
-    try {
-      var config = new PropertiesConfiguration();
-      try (var reader = new InputStreamReader(accumuloConfigUrl.openStream(), UTF_8)) {
-        config.read(reader);
-      }
-      String value = config.getString(propertyName);
-      if (value != null) {
-        return value;
-      }
-      return defaultValue;
-    } catch (Exception e) {
-      throw new IllegalStateException(
-          "Failed to look up property " + propertyName + " in " + accumuloConfigUrl.getFile(), e);
-    }
-  }
-
-  /**
-   * Replace environment variables in the classpath string with their actual value
-   */
-  public static String replaceEnvVars(String classpath, Map<String,String> env) {
-    Pattern envPat = Pattern.compile("\\$[A-Za-z][a-zA-Z0-9_]*");
-    Matcher envMatcher = envPat.matcher(classpath);
-    while (envMatcher.find(0)) {
-      // name comes after the '$'
-      String varName = envMatcher.group().substring(1);
-      String varValue = env.get(varName);
-      if (varValue == null) {
-        varValue = "";
-      }
-      classpath = (classpath.substring(0, envMatcher.start()) + varValue
-          + classpath.substring(envMatcher.end()));
-      envMatcher.reset(classpath);
-    }
-    return classpath;
-  }
-
-  /**
-   * Populate the list of URLs with the items in the classpath string
-   */
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "class path configuration is controlled by admin, not unchecked user input")
-  private static void addUrl(String classpath, ArrayList<URL> urls) throws MalformedURLException {
-    classpath = classpath.trim();
-    if (classpath.isEmpty()) {
-      return;
-    }
-
-    classpath = replaceEnvVars(classpath, System.getenv());
-
-    // Try to make a URI out of the classpath
-    URI uri = null;
-    try {
-      uri = new URI(classpath);
-    } catch (URISyntaxException e) {
-      // Not a valid URI
-    }
-
-    if (uri == null || !uri.isAbsolute()
-        || (uri.getScheme() != null && uri.getScheme().equals("file://"))) {
-      // Then treat this URI as a File.
-      // This checks to see if the url string is a dir if it expand and get all jars in that
-      // directory
-      final File extDir = new File(classpath);
-      if (extDir.isDirectory()) {
-        urls.add(extDir.toURI().toURL());
-      } else {
-        if (extDir.getParentFile() != null) {
-          var pattern = Pattern.compile(extDir.getName());
-          File[] extJars =
-              extDir.getParentFile().listFiles((dir, name) -> pattern.matcher(name).matches());
-          if (extJars != null && extJars.length > 0) {
-            for (File jar : extJars) {
-              urls.add(jar.toURI().toURL());
-            }
-          } else {
-            log.debug("ignoring classpath entry {}", classpath);
-          }
-        } else {
-          log.debug("ignoring classpath entry {}", classpath);
-        }
-      }
-    } else {
-      urls.add(uri.toURL());
-    }
-
-  }
-
-  private static ArrayList<URL> findAccumuloURLs() throws IOException {
-    String cp = getAccumuloProperty(GENERAL_CLASSPATHS, null);
-    if (cp == null) {
-      return new ArrayList<>();
-    }
-    log.warn("'{}' is deprecated but was set to '{}' ", GENERAL_CLASSPATHS, cp);
-    String[] cps = replaceEnvVars(cp, System.getenv()).split(",");
-    ArrayList<URL> urls = new ArrayList<>();
-    for (String classpath : cps) {
-      if (!classpath.startsWith("#")) {
-        addUrl(classpath, urls);
-      }
-    }
-    return urls;
-  }
-
-  public static synchronized ClassLoader getClassLoader() throws IOException {
-    if (classloader == null) {
-      ArrayList<URL> urls = findAccumuloURLs();
-
-      ClassLoader parentClassLoader = ClassLoader.getSystemClassLoader();
-
-      log.debug("Create 2nd tier ClassLoader using URLs: {}", urls);
-      classloader =
-          new URLClassLoader("AccumuloClassLoader (loads everything defined by general.classpaths)",
-              urls.toArray(new URL[urls.size()]), parentClassLoader) {
-            @Override
-            protected synchronized Class<?> loadClass(String name, boolean resolve)
-                throws ClassNotFoundException {
-
-              if (name.startsWith("org.apache.accumulo.start.classloader.vfs")) {
-                Class<?> c = findLoadedClass(name);
-                if (c == null) {
-                  try {
-                    // try finding this class here instead of parent
-                    findClass(name);
-                  } catch (ClassNotFoundException e) {}
-                }
-              }
-              return super.loadClass(name, resolve);
-            }
-          };
-    }
-
-    return classloader;
-  }
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoader.java
deleted file mode 100644
index 4dccc07..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoader.java
+++ /dev/null
@@ -1,330 +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.start.classloader.vfs;
-
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.function.Consumer;
-import java.util.stream.Stream;
-
-import org.apache.commons.vfs2.FileChangeEvent;
-import org.apache.commons.vfs2.FileListener;
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.apache.commons.vfs2.impl.DefaultFileMonitor;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Classloader that delegates operations to a VFSClassLoader object. This class also listens for
- * changes in any of the files/directories that are in the classpath and will recreate the delegate
- * object if there is any change in the classpath.
- */
-@Deprecated
-public class AccumuloReloadingVFSClassLoader implements FileListener, ReloadingClassLoader {
-
-  private static final Logger log = LoggerFactory.getLogger(AccumuloReloadingVFSClassLoader.class);
-
-  // set to 5 mins. The rationale behind this large time is to avoid a gazillion tservers all asking
-  // the name node for info too frequently.
-  private static final long DEFAULT_TIMEOUT = MINUTES.toMillis(5);
-
-  private volatile long maxWaitInterval = 60000;
-
-  private volatile long maxRetries = -1;
-
-  private volatile long sleepInterval = 5000;
-
-  private FileObject[] files;
-  private VFSClassLoader cl;
-  private final ReloadingClassLoader parent;
-  private final String uris;
-  private final DefaultFileMonitor monitor;
-  private final boolean preDelegate;
-  private final ThreadPoolExecutor executor;
-  {
-    BlockingQueue<Runnable> queue = new ArrayBlockingQueue<>(2);
-    ThreadFactory factory = r -> {
-      Thread t = new Thread(r);
-      t.setDaemon(true);
-      return t;
-    };
-    executor = new ThreadPoolExecutor(1, 1, 1, SECONDS, queue, factory);
-  }
-
-  private final Runnable refresher = new Runnable() {
-    @Override
-    public void run() {
-      while (!executor.isTerminating()) {
-        try {
-          FileSystemManager vfs = AccumuloVFSClassLoader.generateVfs();
-          FileObject[] files = AccumuloVFSClassLoader.resolve(vfs, uris);
-
-          long retries = 0;
-          long currentSleepMillis = sleepInterval;
-
-          if (files.length == 0) {
-            while (files.length == 0 && retryPermitted(retries)) {
-
-              try {
-                log.debug("VFS path was empty.  Waiting " + currentSleepMillis + " ms to retry");
-                Thread.sleep(currentSleepMillis);
-
-                files = AccumuloVFSClassLoader.resolve(vfs, uris);
-                retries++;
-
-                currentSleepMillis = Math.min(maxWaitInterval, currentSleepMillis + sleepInterval);
-
-              } catch (InterruptedException e) {
-                log.error("VFS Retry Interrupted", e);
-                throw new RuntimeException(e);
-              }
-
-            }
-
-            // There is a chance that the listener was removed from the top level directory or
-            // its children if they were deleted within some time window. Re-add files to be
-            // monitored. The Monitor will ignore files that are already/still being monitored.
-            // forEachCatchRTEs will capture a stream of thrown exceptions.
-            // and can collect them to list or reduce into one exception
-
-            forEachCatchRTEs(Arrays.stream(files), o -> {
-              addFileToMonitor(o);
-              log.debug("monitoring {}", o);
-            });
-          }
-
-          log.debug("Rebuilding dynamic classloader using files- {}", stringify(files));
-
-          VFSClassLoader cl;
-          if (preDelegate) {
-            cl = new VFSClassLoader(files, vfs, parent.getClassLoader());
-          } else {
-            cl = new PostDelegatingVFSClassLoader(files, vfs, parent.getClassLoader());
-          }
-          updateClassloader(files, cl);
-          return;
-        } catch (Exception e) {
-          log.error("{}", e.getMessage(), e);
-          try {
-            Thread.sleep(DEFAULT_TIMEOUT);
-          } catch (InterruptedException ie) {
-            log.error("{}", ie.getMessage(), ie);
-          }
-        }
-      }
-    }
-  };
-
-  public String stringify(FileObject[] files) {
-    StringBuilder sb = new StringBuilder();
-    sb.append('[');
-    String delim = "";
-    for (FileObject file : files) {
-      sb.append(delim);
-      delim = ", ";
-      sb.append(file.getName());
-    }
-    sb.append(']');
-    return sb.toString();
-  }
-
-  @Override
-  public synchronized ClassLoader getClassLoader() {
-    if (cl.getParent() != parent.getClassLoader()) {
-      scheduleRefresh();
-    }
-    return cl;
-  }
-
-  private void scheduleRefresh() {
-    try {
-      executor.execute(refresher);
-    } catch (RejectedExecutionException e) {
-      log.trace("Ignoring refresh request (already refreshing)");
-    }
-  }
-
-  private synchronized void updateClassloader(FileObject[] files, VFSClassLoader cl) {
-    this.files = files;
-    this.cl = cl;
-  }
-
-  public AccumuloReloadingVFSClassLoader(String uris, FileSystemManager vfs,
-      ReloadingClassLoader parent, long monitorDelay, boolean preDelegate)
-      throws FileSystemException {
-
-    this.uris = uris;
-    this.parent = parent;
-    this.preDelegate = preDelegate;
-
-    ArrayList<FileObject> pathsToMonitor = new ArrayList<>();
-    files = AccumuloVFSClassLoader.resolve(vfs, uris, pathsToMonitor);
-
-    if (preDelegate) {
-      cl = new VFSClassLoader(files, vfs, parent.getClassLoader()) {
-        @Override
-        public String getName() {
-          return "AccumuloReloadingVFSClassLoader (loads everything defined by general.dynamic.classpaths)";
-        }
-      };
-    } else {
-      cl = new PostDelegatingVFSClassLoader(files, vfs, parent.getClassLoader()) {
-        @Override
-        public String getName() {
-          return "AccumuloReloadingVFSClassLoader (loads everything defined by general.dynamic.classpaths)";
-        }
-      };
-    }
-
-    monitor = new DefaultFileMonitor(this);
-    monitor.setDelay(monitorDelay);
-    monitor.setRecursive(false);
-
-    forEachCatchRTEs(pathsToMonitor.stream(), o -> {
-      addFileToMonitor(o);
-      log.debug("monitoring {}", o);
-    });
-
-    monitor.start();
-  }
-
-  private void addFileToMonitor(FileObject file) throws RuntimeException {
-    try {
-      if (monitor != null) {
-        monitor.addFile(file);
-      }
-    } catch (RuntimeException re) {
-      if (re.getMessage().contains("files-cache")) {
-        log.error("files-cache error adding {} to VFS monitor. "
-            + "There is no implementation for files-cache in VFS2", file, re);
-      } else {
-        log.error("Runtime error adding {} to VFS monitor", file, re);
-      }
-
-      throw re;
-    }
-  }
-
-  private void removeFile(FileObject file) throws RuntimeException {
-    try {
-      if (monitor != null) {
-        monitor.removeFile(file);
-      }
-    } catch (RuntimeException re) {
-      log.error("Error removing file from VFS cache {}", file, re);
-      throw re;
-    }
-  }
-
-  public static <T> void forEachCatchRTEs(Stream<T> stream, Consumer<T> consumer) {
-    stream.flatMap(o -> {
-      try {
-        consumer.accept(o);
-        return null;
-      } catch (RuntimeException e) {
-        return Stream.of(e);
-      }
-    }).reduce((e1, e2) -> {
-      e1.addSuppressed(e2);
-      return e1;
-    }).ifPresent(e -> {
-      throw e;
-    });
-  }
-
-  public AccumuloReloadingVFSClassLoader(String uris, FileSystemManager vfs,
-      final ReloadingClassLoader parent, boolean preDelegate) throws FileSystemException {
-    this(uris, vfs, parent, DEFAULT_TIMEOUT, preDelegate);
-  }
-
-  /**
-   * Should be ok if this is not called because the thread started by DefaultFileMonitor is a daemon
-   * thread
-   */
-  public void close() {
-
-    forEachCatchRTEs(Stream.of(files), o -> {
-      removeFile(o);
-      log.debug("Removing file from monitoring {}", o);
-    });
-
-    executor.shutdownNow();
-    monitor.stop();
-  }
-
-  @Override
-  public void fileCreated(FileChangeEvent event) throws Exception {
-    if (log.isDebugEnabled()) {
-      log.debug("{} created, recreating classloader", event.getFileObject().getURL());
-    }
-    scheduleRefresh();
-  }
-
-  @Override
-  public void fileDeleted(FileChangeEvent event) throws Exception {
-    if (log.isDebugEnabled()) {
-      log.debug("{} deleted, recreating classloader", event.getFileObject().getURL());
-    }
-    scheduleRefresh();
-  }
-
-  @Override
-  public void fileChanged(FileChangeEvent event) throws Exception {
-    if (log.isDebugEnabled()) {
-      log.debug("{} changed, recreating classloader", event.getFileObject().getURL());
-    }
-    scheduleRefresh();
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buf = new StringBuilder();
-
-    for (FileObject f : files) {
-      try {
-        buf.append("\t").append(f.getURL()).append("\n");
-      } catch (FileSystemException e) {
-        log.error("Error getting URL for file", e);
-      }
-    }
-
-    return buf.toString();
-  }
-
-  // VisibleForTesting intentionally not using annotation from Guava because it adds unwanted
-  // dependency
-  void setMaxRetries(long maxRetries) {
-    this.maxRetries = maxRetries;
-  }
-
-  private boolean retryPermitted(long retries) {
-    return (maxRetries < 0 || retries < maxRetries);
-  }
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
deleted file mode 100644
index d762602..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
+++ /dev/null
@@ -1,458 +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.start.classloader.vfs;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.lang.management.ManagementFactory;
-import java.lang.ref.WeakReference;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.regex.Pattern;
-
-import org.apache.accumulo.start.classloader.AccumuloClassLoader;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.vfs2.CacheStrategy;
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.apache.commons.vfs2.FileType;
-import org.apache.commons.vfs2.cache.SoftRefFilesCache;
-import org.apache.commons.vfs2.impl.DefaultFileSystemManager;
-import org.apache.commons.vfs2.impl.FileContentInfoFilenameFactory;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-import org.apache.commons.vfs2.provider.FileReplicator;
-import org.apache.commons.vfs2.provider.hdfs.HdfsFileObject;
-import org.apache.commons.vfs2.provider.hdfs.HdfsFileProvider;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This class builds a hierarchy of Classloaders in the form of:
- *
- * <pre>
- * SystemClassLoader that loads JVM classes
- *       ^
- *       |
- * AccumuloClassLoader loads jars from locations in general.classpaths.
- * Usually the URLs for HADOOP_HOME, ZOOKEEPER_HOME, ACCUMULO_HOME and their associated directories
- *       ^
- *       |
- * VFSClassLoader that loads jars from locations in general.vfs.classpaths.
- * Can be used to load accumulo jar from HDFS
- *       ^
- *       |
- * AccumuloReloadingVFSClassLoader That loads jars from locations in general.dynamic.classpaths.
- * Used to load jar dynamically.
- * </pre>
- */
-@Deprecated
-public class AccumuloVFSClassLoader {
-
-  public static class AccumuloVFSClassLoaderShutdownThread implements Runnable {
-
-    @Override
-    public void run() {
-      try {
-        AccumuloVFSClassLoader.close();
-      } catch (Exception e) {
-        // do nothing, we are shutting down anyway
-      }
-    }
-
-  }
-
-  private static List<WeakReference<DefaultFileSystemManager>> vfsInstances =
-      Collections.synchronizedList(new ArrayList<>());
-
-  public static final String DYNAMIC_CLASSPATH_PROPERTY_NAME = "general.dynamic.classpaths";
-
-  public static final String DEFAULT_DYNAMIC_CLASSPATH_VALUE = "";
-
-  public static final String VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY = "general.vfs.classpaths";
-
-  public static final String VFS_CONTEXT_CLASSPATH_PROPERTY = "general.vfs.context.classpath.";
-
-  public static final String VFS_CACHE_DIR = "general.vfs.cache.dir";
-
-  private static ClassLoader parent = null;
-  private static volatile ReloadingClassLoader loader = null;
-  private static final Object lock = new Object();
-
-  private static ContextManager contextManager;
-
-  private static final Logger log = LoggerFactory.getLogger(AccumuloVFSClassLoader.class);
-
-  static {
-    // Register the shutdown hook
-    Runtime.getRuntime().addShutdownHook(new Thread(new AccumuloVFSClassLoaderShutdownThread()));
-  }
-
-  static FileObject[] resolve(FileSystemManager vfs, String uris) throws FileSystemException {
-    return resolve(vfs, uris, new ArrayList<>());
-  }
-
-  static FileObject[] resolve(FileSystemManager vfs, String uris,
-      ArrayList<FileObject> pathsToMonitor) throws FileSystemException {
-    if (uris == null) {
-      return new FileObject[0];
-    }
-
-    ArrayList<FileObject> classpath = new ArrayList<>();
-
-    pathsToMonitor.clear();
-
-    for (String path : uris.split(",")) {
-
-      path = path.trim();
-
-      if (path.equals("")) {
-        continue;
-      }
-
-      path = AccumuloClassLoader.replaceEnvVars(path, System.getenv());
-
-      log.debug("Resolving path element: {}", path);
-      FileObject fo = vfs.resolveFile(path);
-
-      switch (fo.getType()) {
-        case FILE:
-        case FOLDER:
-          classpath.add(fo);
-          pathsToMonitor.add(fo);
-          break;
-        case IMAGINARY:
-          // assume it's a pattern
-          var pattern = Pattern.compile(fo.getName().getBaseName());
-          if (fo.getParent() != null) {
-            // still monitor the parent
-            pathsToMonitor.add(fo.getParent());
-            if (fo.getParent().getType() == FileType.FOLDER) {
-              FileObject[] children = fo.getParent().getChildren();
-              for (FileObject child : children) {
-                if (child.getType() == FileType.FILE
-                    && pattern.matcher(child.getName().getBaseName()).matches()) {
-                  classpath.add(child);
-                }
-              }
-            } else {
-              log.debug("classpath entry " + fo.getParent() + " is " + fo.getParent().getType());
-            }
-          } else {
-            log.warn("ignoring classpath entry {}", fo);
-          }
-          break;
-        default:
-          log.warn("ignoring classpath entry {}", fo);
-          break;
-      }
-
-    }
-
-    return classpath.toArray(new FileObject[classpath.size()]);
-  }
-
-  private static ReloadingClassLoader createDynamicClassloader(final ClassLoader parent)
-      throws IOException {
-    String dynamicCPath = AccumuloClassLoader.getAccumuloProperty(DYNAMIC_CLASSPATH_PROPERTY_NAME,
-        DEFAULT_DYNAMIC_CLASSPATH_VALUE);
-
-    ReloadingClassLoader wrapper = () -> parent;
-
-    if (dynamicCPath == null || dynamicCPath.equals("")) {
-      return wrapper;
-    }
-
-    // TODO monitor time for lib/ext was 1 sec... should this be configurable? - ACCUMULO-1301
-    return new AccumuloReloadingVFSClassLoader(dynamicCPath, generateVfs(), wrapper, 1000, true);
-  }
-
-  public static ClassLoader getClassLoader() {
-    try {
-      return getClassLoader_Internal();
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
-
-  private static ClassLoader getClassLoader_Internal() throws IOException {
-    ReloadingClassLoader localLoader = loader;
-    while (localLoader == null) {
-      synchronized (lock) {
-        if (loader == null) {
-
-          FileSystemManager vfs = generateVfs();
-
-          // Set up the 2nd tier class loader
-          if (parent == null) {
-            parent = AccumuloClassLoader.getClassLoader();
-          }
-
-          FileObject[] vfsCP = resolve(vfs, AccumuloClassLoader
-              .getAccumuloProperty(VFS_CLASSLOADER_SYSTEM_CLASSPATH_PROPERTY, ""));
-
-          if (vfsCP.length == 0) {
-            localLoader = createDynamicClassloader(parent);
-            loader = localLoader;
-            return localLoader.getClassLoader();
-          }
-
-          // Create the Accumulo Context ClassLoader using the DEFAULT_CONTEXT
-          localLoader = createDynamicClassloader(new VFSClassLoader(vfsCP, vfs, parent));
-          loader = localLoader;
-
-          // An HDFS FileSystem and Configuration object were created for each unique HDFS namespace
-          // in the call to resolve above.
-          // The HDFS Client did us a favor and cached these objects so that the next time someone
-          // calls FileSystem.get(uri), they
-          // get the cached object. However, these objects were created not with the system VFS
-          // classloader, but the classloader above
-          // it. We need to override the classloader on the Configuration objects. Ran into an issue
-          // were log recovery was being attempted
-          // and SequenceFile$Reader was trying to instantiate the key class via
-          // WritableName.getClass(String, Configuration)
-          for (FileObject fo : vfsCP) {
-            if (fo instanceof HdfsFileObject) {
-              String uri = fo.getName().getRootURI();
-              Configuration c = new Configuration(true);
-              c.set(FileSystem.FS_DEFAULT_NAME_KEY, uri);
-              FileSystem fs = FileSystem.get(c);
-              fs.getConf().setClassLoader(loader.getClassLoader());
-            }
-          }
-
-        }
-      }
-    }
-
-    return localLoader.getClassLoader();
-  }
-
-  public static FileSystemManager generateVfs() throws FileSystemException {
-    DefaultFileSystemManager vfs = new DefaultFileSystemManager();
-    vfs.addProvider("res", new org.apache.commons.vfs2.provider.res.ResourceFileProvider());
-    vfs.addProvider("zip", new org.apache.commons.vfs2.provider.zip.ZipFileProvider());
-    vfs.addProvider("gz", new org.apache.commons.vfs2.provider.gzip.GzipFileProvider());
-    vfs.addProvider("ram", new org.apache.commons.vfs2.provider.ram.RamFileProvider());
-    vfs.addProvider("file", new org.apache.commons.vfs2.provider.local.DefaultLocalFileProvider());
-    vfs.addProvider("jar", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-    vfs.addProvider("http", new org.apache.commons.vfs2.provider.http.HttpFileProvider());
-    vfs.addProvider("https", new org.apache.commons.vfs2.provider.https.HttpsFileProvider());
-    vfs.addProvider("ftp", new org.apache.commons.vfs2.provider.ftp.FtpFileProvider());
-    vfs.addProvider("ftps", new org.apache.commons.vfs2.provider.ftps.FtpsFileProvider());
-    vfs.addProvider("war", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-    vfs.addProvider("par", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-    vfs.addProvider("ear", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-    vfs.addProvider("sar", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-    vfs.addProvider("ejb3", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-    vfs.addProvider("tmp", new org.apache.commons.vfs2.provider.temp.TemporaryFileProvider());
-    vfs.addProvider("tar", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
-    vfs.addProvider("tbz2", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
-    vfs.addProvider("tgz", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
-    vfs.addProvider("bz2", new org.apache.commons.vfs2.provider.bzip2.Bzip2FileProvider());
-    vfs.addProvider("hdfs", new HdfsFileProvider());
-    vfs.addExtensionMap("jar", "jar");
-    vfs.addExtensionMap("zip", "zip");
-    vfs.addExtensionMap("gz", "gz");
-    vfs.addExtensionMap("tar", "tar");
-    vfs.addExtensionMap("tbz2", "tar");
-    vfs.addExtensionMap("tgz", "tar");
-    vfs.addExtensionMap("bz2", "bz2");
-    vfs.addMimeTypeMap("application/java-archive", "jar");
-    vfs.addMimeTypeMap("application/x-tar", "tar");
-    vfs.addMimeTypeMap("application/x-gzip", "gz");
-    vfs.addMimeTypeMap("application/zip", "zip");
-    vfs.setFileContentInfoFactory(new FileContentInfoFilenameFactory());
-    vfs.setFilesCache(new SoftRefFilesCache());
-    File cacheDir = computeTopCacheDir();
-    vfs.setReplicator(new UniqueFileReplicator(cacheDir));
-    vfs.setCacheStrategy(CacheStrategy.ON_RESOLVE);
-    vfs.init();
-    vfsInstances.add(new WeakReference<>(vfs));
-    return vfs;
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "tmpdir is controlled by admin, not unchecked user input")
-  private static File computeTopCacheDir() {
-    String cacheDirPath = AccumuloClassLoader.getAccumuloProperty(VFS_CACHE_DIR,
-        System.getProperty("java.io.tmpdir"));
-    String procName = ManagementFactory.getRuntimeMXBean().getName();
-    return new File(cacheDirPath,
-        "accumulo-vfs-cache-" + procName + "-" + System.getProperty("user.name", "nouser"));
-  }
-
-  public interface Printer {
-    void print(String s);
-  }
-
-  public static void printClassPath(boolean debug) {
-    printClassPath(System.out::print, debug);
-  }
-
-  public static String getClassPath(boolean debug) {
-    StringBuilder cp = new StringBuilder();
-    printClassPath(cp::append, debug);
-    return cp.toString();
-  }
-
-  private static void printJar(Printer out, String jarPath, boolean debug, boolean sawFirst) {
-    if (debug) {
-      out.print("\t");
-    }
-    if (!debug && sawFirst) {
-      out.print(":");
-    }
-    out.print(jarPath);
-    if (debug) {
-      out.print("\n");
-    }
-  }
-
-  public static void printClassPath(Printer out, boolean debug) {
-    try {
-      ClassLoader cl = getClassLoader();
-      ArrayList<ClassLoader> classloaders = new ArrayList<>();
-
-      while (cl != null) {
-        classloaders.add(cl);
-        cl = cl.getParent();
-      }
-
-      Collections.reverse(classloaders);
-
-      int level = 0;
-
-      for (ClassLoader classLoader : classloaders) {
-
-        level++;
-
-        if (debug && level > 1) {
-          out.print("\n");
-        }
-        if (!debug && level < 2) {
-          continue;
-        }
-
-        boolean sawFirst = false;
-        String classLoaderDescription = "Level: " + level + ", Name: " + classLoader.getName()
-            + ", class: " + classLoader.getClass().getName();
-        if (classLoader.getClass().getName().startsWith("jdk.internal")) {
-          if (debug) {
-            out.print(classLoaderDescription + ": configuration not inspectable.\n");
-          }
-        } else if (classLoader instanceof URLClassLoader) {
-          if (debug) {
-            out.print(classLoaderDescription + ": URL classpath items are:\n");
-          }
-          for (URL u : ((URLClassLoader) classLoader).getURLs()) {
-            printJar(out, u.getFile(), debug, sawFirst);
-            sawFirst = true;
-          }
-        } else if (classLoader instanceof VFSClassLoader) {
-          if (debug) {
-            out.print(classLoaderDescription + ": VFS classpaths items are:\n");
-          }
-          VFSClassLoader vcl = (VFSClassLoader) classLoader;
-          for (FileObject f : vcl.getFileObjects()) {
-            printJar(out, f.getURL().getFile(), debug, sawFirst);
-            sawFirst = true;
-          }
-        } else {
-          if (debug) {
-            out.print(
-                classLoaderDescription + ": Unknown classloader: " + classLoader.getClass() + "\n");
-          }
-        }
-      }
-      out.print("\n");
-    } catch (Exception t) {
-      throw new RuntimeException(t);
-    }
-  }
-
-  public static void setContextConfig(Supplier<Map<String,String>> contextConfigSupplier) {
-    var config = new ContextManager.DefaultContextsConfig(contextConfigSupplier);
-    try {
-      getContextManager().setContextConfig(config);
-    } catch (IOException e) {
-      throw new UncheckedIOException(e);
-    }
-  }
-
-  public static void removeUnusedContexts(Set<String> contextsInUse) {
-    try {
-      getContextManager().removeUnusedContexts(contextsInUse);
-    } catch (IOException e) {
-      log.warn("{}", e.getMessage(), e);
-    }
-  }
-
-  public static ClassLoader getContextClassLoader(String contextName) {
-    try {
-      return getContextManager().getClassLoader(contextName);
-    } catch (IOException e) {
-      throw new UncheckedIOException(
-          "Error getting context class loader for context: " + contextName, e);
-    }
-  }
-
-  public static synchronized ContextManager getContextManager() throws IOException {
-    if (contextManager == null) {
-      getClassLoader();
-      contextManager = new ContextManager(generateVfs(), AccumuloVFSClassLoader::getClassLoader);
-    }
-
-    return contextManager;
-  }
-
-  public static void close() {
-    for (WeakReference<DefaultFileSystemManager> vfsInstance : vfsInstances) {
-      DefaultFileSystemManager ref = vfsInstance.get();
-      if (ref != null) {
-        FileReplicator replicator;
-        try {
-          replicator = ref.getReplicator();
-          if (replicator instanceof UniqueFileReplicator) {
-            ((UniqueFileReplicator) replicator).close();
-          }
-        } catch (FileSystemException e) {
-          log.error("FileSystemException", e);
-        }
-        ref.close();
-      }
-    }
-    try {
-      FileUtils.deleteDirectory(computeTopCacheDir());
-    } catch (IOException e) {
-      log.error("IOException", e);
-    }
-  }
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/ContextManager.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/ContextManager.java
deleted file mode 100644
index c57c38c..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/ContextManager.java
+++ /dev/null
@@ -1,233 +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.start.classloader.vfs;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.function.Supplier;
-
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-class ContextManager {
-
-  private static final Logger log = LoggerFactory.getLogger(ContextManager.class);
-
-  // there is a lock per context so that one context can initialize w/o blocking another context
-  private class Context {
-    AccumuloReloadingVFSClassLoader loader;
-    ContextConfig cconfig;
-    boolean closed = false;
-
-    Context(ContextConfig cconfig) {
-      this.cconfig = cconfig;
-    }
-
-    synchronized ClassLoader getClassLoader() throws FileSystemException {
-      if (closed) {
-        return null;
-      }
-
-      if (loader == null) {
-        log.debug(
-            "ClassLoader not created for context {}, creating new one. uris: {}, preDelegation: {}",
-            cconfig.name, cconfig.uris, cconfig.preDelegation);
-        loader =
-            new AccumuloReloadingVFSClassLoader(cconfig.uris, vfs, parent, cconfig.preDelegation);
-      }
-
-      return loader.getClassLoader();
-    }
-
-    synchronized void close() {
-      closed = true;
-      if (loader != null) {
-        loader.close();
-      }
-      loader = null;
-    }
-  }
-
-  private Map<String,Context> contexts = new HashMap<>();
-
-  private volatile ContextsConfig config;
-  private FileSystemManager vfs;
-  private ReloadingClassLoader parent;
-
-  ContextManager(FileSystemManager vfs, ReloadingClassLoader parent) {
-    this.vfs = vfs;
-    this.parent = parent;
-  }
-
-  @Deprecated
-  public static class ContextConfig {
-    final String name;
-    final String uris;
-    final boolean preDelegation;
-
-    public ContextConfig(String name, String uris, boolean preDelegation) {
-      this.name = name;
-      this.uris = uris;
-      this.preDelegation = preDelegation;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o instanceof ContextConfig) {
-        ContextConfig oc = (ContextConfig) o;
-
-        return name.equals(oc.name) && uris.equals(oc.uris) && preDelegation == oc.preDelegation;
-      }
-
-      return false;
-    }
-
-    @Override
-    public int hashCode() {
-      return name.hashCode() + uris.hashCode()
-          + (preDelegation ? Boolean.TRUE : Boolean.FALSE).hashCode();
-    }
-  }
-
-  public interface ContextsConfig {
-    ContextConfig getContextConfig(String context);
-  }
-
-  public static class DefaultContextsConfig implements ContextsConfig {
-
-    private final Supplier<Map<String,String>> vfsContextClasspathPropertiesProvider;
-
-    public DefaultContextsConfig(
-        Supplier<Map<String,String>> vfsContextClasspathPropertiesProvider) {
-      this.vfsContextClasspathPropertiesProvider = vfsContextClasspathPropertiesProvider;
-    }
-
-    @Override
-    public ContextConfig getContextConfig(String context) {
-
-      String prop = AccumuloVFSClassLoader.VFS_CONTEXT_CLASSPATH_PROPERTY + context;
-      Map<String,String> props = vfsContextClasspathPropertiesProvider.get();
-
-      String uris = props.get(prop);
-
-      if (uris == null) {
-        return null;
-      }
-
-      String delegate = props.get(prop + ".delegation");
-
-      boolean preDelegate = true;
-
-      if (delegate != null && delegate.trim().equalsIgnoreCase("post")) {
-        preDelegate = false;
-      }
-
-      return new ContextConfig(context, uris, preDelegate);
-    }
-  }
-
-  /**
-   * configuration must be injected for ContextManager to work
-   */
-  public synchronized void setContextConfig(ContextsConfig config) {
-    if (this.config != null) {
-      throw new IllegalStateException("Context manager config already set");
-    }
-    this.config = config;
-  }
-
-  public ClassLoader getClassLoader(String contextName) throws FileSystemException {
-
-    ContextConfig cconfig = config.getContextConfig(contextName);
-
-    if (cconfig == null) {
-      throw new IllegalArgumentException("Unknown context " + contextName);
-    }
-
-    Context context = null;
-    Context contextToClose = null;
-
-    synchronized (this) {
-      // only manipulate internal data structs in this sync block... avoid creating or closing
-      // classloader, reading config, etc... basically avoid operations
-      // that may block
-      context = contexts.get(contextName);
-
-      if (context == null) {
-        context = new Context(cconfig);
-        contexts.put(contextName, context);
-      } else if (!context.cconfig.equals(cconfig)) {
-        contextToClose = context;
-        context = new Context(cconfig);
-        contexts.put(contextName, context);
-      }
-    }
-
-    if (contextToClose != null) {
-      contextToClose.close();
-    }
-
-    ClassLoader loader = context.getClassLoader();
-    if (loader == null) {
-      // oops, context was closed by another thread, try again
-      ClassLoader loader2 = getClassLoader(contextName);
-      log.debug("Returning new classloader {} for context {}", loader2.getClass().getName(),
-          contextName);
-      return loader2;
-    }
-
-    log.debug("Returning classloader {} for context {}", loader.getClass().getName(), contextName);
-    return loader;
-
-  }
-
-  public <U> Class<? extends U> loadClass(String context, String classname, Class<U> extension)
-      throws ClassNotFoundException {
-    try {
-      return getClassLoader(context).loadClass(classname).asSubclass(extension);
-    } catch (IOException e) {
-      throw new ClassNotFoundException("IO Error loading class " + classname, e);
-    }
-  }
-
-  public void removeUnusedContexts(Set<String> configuredContexts) {
-
-    Map<String,Context> unused;
-
-    // ContextManager knows of some set of contexts. This method will be called with
-    // the set of currently configured contexts. We will close the contexts that are
-    // no longer in the configuration.
-    synchronized (this) {
-      unused = new HashMap<>(contexts);
-      unused.keySet().removeAll(configuredContexts);
-      contexts.keySet().removeAll(unused.keySet());
-    }
-    for (Entry<String,Context> e : unused.entrySet()) {
-      // close outside of lock
-      log.info("Closing unused context: {}", e.getKey());
-      e.getValue().close();
-    }
-  }
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/PostDelegatingVFSClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/PostDelegatingVFSClassLoader.java
deleted file mode 100644
index 9d0b30c..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/PostDelegatingVFSClassLoader.java
+++ /dev/null
@@ -1,50 +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.start.classloader.vfs;
-
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-
-@Deprecated
-public class PostDelegatingVFSClassLoader extends VFSClassLoader {
-
-  public PostDelegatingVFSClassLoader(FileObject[] files, FileSystemManager manager,
-      ClassLoader parent) throws FileSystemException {
-    super(files, manager, parent);
-  }
-
-  @Override
-  protected synchronized Class<?> loadClass(String name, boolean resolve)
-      throws ClassNotFoundException {
-    Class<?> c = findLoadedClass(name);
-    if (c != null) {
-      return c;
-    }
-    try {
-      // try finding this class here instead of parent
-      return findClass(name);
-    } catch (ClassNotFoundException e) {
-
-    }
-    return super.loadClass(name, resolve);
-  }
-
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/ReloadingClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/ReloadingClassLoader.java
deleted file mode 100644
index b6125ae..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/ReloadingClassLoader.java
+++ /dev/null
@@ -1,24 +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.start.classloader.vfs;
-
-@Deprecated
-public interface ReloadingClassLoader {
-  ClassLoader getClassLoader();
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java
deleted file mode 100644
index c2e0881..0000000
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.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.start.classloader.vfs;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.FileSelector;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.provider.FileReplicator;
-import org.apache.commons.vfs2.provider.UriParser;
-import org.apache.commons.vfs2.provider.VfsComponent;
-import org.apache.commons.vfs2.provider.VfsComponentContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated
-public class UniqueFileReplicator implements VfsComponent, FileReplicator {
-
-  private static final char[] TMP_RESERVED_CHARS =
-      {'?', '/', '\\', ' ', '&', '"', '\'', '*', '#', ';', ':', '<', '>', '|'};
-  private static final Logger log = LoggerFactory.getLogger(UniqueFileReplicator.class);
-
-  private File tempDir;
-  private VfsComponentContext context;
-  private List<File> tmpFiles = Collections.synchronizedList(new ArrayList<>());
-
-  public UniqueFileReplicator(File tempDir) {
-    this.tempDir = tempDir;
-    if (!tempDir.exists() && !tempDir.mkdirs()) {
-      log.warn("Unexpected error creating directory {}", tempDir);
-    }
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "input files are specified by admin, not unchecked user input")
-  @Override
-  public File replicateFile(FileObject srcFile, FileSelector selector) throws FileSystemException {
-    String baseName = srcFile.getName().getBaseName();
-
-    try {
-      String safeBasename = UriParser.encode(baseName, TMP_RESERVED_CHARS).replace('%', '_');
-      File file = File.createTempFile("vfsr_", "_" + safeBasename, tempDir);
-      file.deleteOnExit();
-
-      final FileObject destFile = context.toFileObject(file);
-      destFile.copyFrom(srcFile, selector);
-
-      return file;
-    } catch (IOException e) {
-      throw new FileSystemException(e);
-    }
-  }
-
-  @Override
-  public void setLogger(Log logger) {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
-  public void setContext(VfsComponentContext context) {
-    this.context = context;
-  }
-
-  @Override
-  public void init() throws FileSystemException {
-
-  }
-
-  @Override
-  public void close() {
-    synchronized (tmpFiles) {
-      for (File tmpFile : tmpFiles) {
-        if (!tmpFile.delete()) {
-          log.warn("File does not exist: {}", tmpFile);
-        }
-      }
-    }
-
-    if (tempDir.exists()) {
-      String[] list = tempDir.list();
-      int numChildren = list == null ? 0 : list.length;
-      if (numChildren == 0 && !tempDir.delete()) {
-        log.warn("Cannot delete empty directory: {}", tempDir);
-      }
-    }
-  }
-}
diff --git a/start/src/main/java/org/apache/accumulo/start/spi/KeywordExecutable.java b/start/src/main/java/org/apache/accumulo/start/spi/KeywordExecutable.java
index 83015a7..4f67f5f 100644
--- a/start/src/main/java/org/apache/accumulo/start/spi/KeywordExecutable.java
+++ b/start/src/main/java/org/apache/accumulo/start/spi/KeywordExecutable.java
@@ -34,7 +34,7 @@
  *
  * <p>
  * One way to easily create META-INF/services files is to use the
- * <a href="https://github.com/google/auto/tree/master/service">AutoService</a> annotation.
+ * <a href="https://github.com/google/auto/tree/main/service">AutoService</a> annotation.
  *
  * <p>
  * It generally should be avoided, but if the implementing class also must have a redundant main
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/MiniDFSUtil.java b/start/src/main/java/org/apache/accumulo/start/util/MiniDFSUtil.java
similarity index 91%
rename from start/src/main/java/org/apache/accumulo/start/classloader/vfs/MiniDFSUtil.java
rename to start/src/main/java/org/apache/accumulo/start/util/MiniDFSUtil.java
index 59288a5..7a11f2a 100644
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/MiniDFSUtil.java
+++ b/start/src/main/java/org/apache/accumulo/start/util/MiniDFSUtil.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.start.classloader.vfs;
+package org.apache.accumulo.start.util;
 
 import java.io.BufferedReader;
 import java.io.IOException;
@@ -45,8 +45,8 @@
 
         return String.format("%03o", newPermission);
       }
-    } catch (Exception e) {
-      throw new RuntimeException("Error getting umask from O/S", e);
+    } catch (IOException | InterruptedException e) {
+      throw new IllegalStateException("Error getting umask from O/S", e);
     }
   }
 
diff --git a/start/src/main/spotbugs/exclude-filter.xml b/start/src/main/spotbugs/exclude-filter.xml
index c5b24cd..358f5e7 100644
--- a/start/src/main/spotbugs/exclude-filter.xml
+++ b/start/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>
     <!-- More convenient to ignore these everywhere, because it's very common and unimportant -->
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java
deleted file mode 100644
index 8bed40b..0000000
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloClasspathTest.java
+++ /dev/null
@@ -1,58 +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.start.classloader.vfs;
-
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.regex.Pattern;
-
-import org.junit.jupiter.api.Test;
-
-public class AccumuloClasspathTest {
-
-  private static void assertPattern(String output, Pattern pattern, boolean shouldMatch) {
-    if (shouldMatch) {
-      assertTrue(pattern.matcher(output).matches(),
-          "Pattern " + pattern + " did not match output: " + output);
-    } else {
-      assertFalse(pattern.matcher(output).matches(),
-          "Pattern " + pattern + " should not match output: " + output);
-    }
-  }
-
-  @Test
-  public void basic() {
-    var pattern = Pattern.compile("(?s).*\\s+.*\\n$");
-    assertPattern(getClassPath(true), pattern, true);
-    assertTrue(getClassPath(true).contains("app"));
-    assertTrue(getClassPath(true).contains("Level"));
-
-    assertTrue(getClassPath(true).length() > getClassPath(false).length());
-
-    assertPattern(getClassPath(false), pattern, false);
-    assertFalse(getClassPath(false).contains("app"));
-    assertFalse(getClassPath(false).contains("Level"));
-  }
-
-  @SuppressWarnings("deprecation")
-  private String getClassPath(boolean b) {
-    return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.getClassPath(b);
-  }
-}
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java
deleted file mode 100644
index 7e9df6e..0000000
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloReloadingVFSClassLoaderTest.java
+++ /dev/null
@@ -1,188 +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.start.classloader.vfs;
-
-import static org.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertNotSame;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.util.Objects;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated
-@SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-public class AccumuloReloadingVFSClassLoaderTest extends WithTestNames {
-
-  @TempDir
-  private static File folder1;
-
-  private File tmpDir;
-
-  String folderPath;
-  private FileSystemManager vfs;
-
-  @BeforeEach
-  public void setup() throws Exception {
-    tmpDir = new File(folder1, testName());
-
-    vfs = ContextManagerTest.getVFS();
-    folderPath = tmpDir.toURI() + "/.*";
-    FileUtils.copyURLToFile(Objects.requireNonNull(this.getClass().getResource("/HelloWorld.jar")),
-        new File(tmpDir, "HelloWorld.jar"));
-  }
-
-  FileObject[] createFileSystems(FileObject[] fos) throws FileSystemException {
-    FileObject[] rfos = new FileObject[fos.length];
-    for (int i = 0; i < fos.length; i++) {
-      if (vfs.canCreateFileSystem(fos[i])) {
-        rfos[i] = vfs.createFileSystem(fos[i]);
-      } else {
-        rfos[i] = fos[i];
-      }
-    }
-
-    return rfos;
-  }
-
-  @Test
-  public void testConstructor() throws Exception {
-    FileObject testDir = vfs.resolveFile(tmpDir.toURI().toString());
-    FileObject[] dirContents = testDir.getChildren();
-
-    AccumuloReloadingVFSClassLoader arvcl = new AccumuloReloadingVFSClassLoader(folderPath, vfs,
-        ClassLoader::getSystemClassLoader, true);
-
-    VFSClassLoader cl = (VFSClassLoader) arvcl.getClassLoader();
-
-    FileObject[] files = cl.getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents), files);
-
-    arvcl.close();
-  }
-
-  @Test
-  public void testReloading() throws Exception {
-    FileObject testDir = vfs.resolveFile(tmpDir.toURI().toString());
-    FileObject[] dirContents = testDir.getChildren();
-
-    AccumuloReloadingVFSClassLoader arvcl = new AccumuloReloadingVFSClassLoader(folderPath, vfs,
-        ClassLoader::getSystemClassLoader, 1000, true);
-
-    FileObject[] files = ((VFSClassLoader) arvcl.getClassLoader()).getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents), files);
-
-    // set retry settings sufficiently low that not everything is reloaded in the first round
-    arvcl.setMaxRetries(1);
-
-    Class<?> clazz1 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o1 = clazz1.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o1.toString());
-
-    // Check that the class is the same before the update
-    Class<?> clazz1_5 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    assertEquals(clazz1, clazz1_5);
-
-    assertTrue(new File(tmpDir, "HelloWorld.jar").delete());
-
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-
-    // Update the class
-    FileUtils.copyURLToFile(Objects.requireNonNull(this.getClass().getResource("/HelloWorld.jar")),
-        new File(tmpDir, "HelloWorld2.jar"));
-
-    // Wait for the monitor to notice
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-
-    Class<?> clazz2 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o2 = clazz2.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o2.toString());
-
-    // This is false because they are loaded by a different classloader
-    assertNotEquals(clazz1, clazz2);
-    assertNotEquals(o1, o2);
-
-    arvcl.close();
-  }
-
-  @Test
-  public void testReloadingWithLongerTimeout() throws Exception {
-    FileObject testDir = vfs.resolveFile(tmpDir.toURI().toString());
-    FileObject[] dirContents = testDir.getChildren();
-
-    AccumuloReloadingVFSClassLoader arvcl = new AccumuloReloadingVFSClassLoader(folderPath, vfs,
-        ClassLoader::getSystemClassLoader, 1000, true);
-
-    FileObject[] files = ((VFSClassLoader) arvcl.getClassLoader()).getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents), files);
-
-    // set retry settings sufficiently high such that reloading happens in the first rounds
-    arvcl.setMaxRetries(3);
-
-    Class<?> clazz1 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o1 = clazz1.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o1.toString());
-
-    // Check that the class is the same before the update
-    Class<?> clazz1_5 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    assertEquals(clazz1, clazz1_5);
-
-    assertTrue(new File(tmpDir, "HelloWorld.jar").delete());
-
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-
-    // Update the class
-    FileUtils.copyURLToFile(Objects.requireNonNull(this.getClass().getResource("/HelloWorld.jar")),
-        new File(tmpDir, "HelloWorld2.jar"));
-
-    // Wait for the monitor to notice
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-
-    Class<?> clazz2 = arvcl.getClassLoader().loadClass("test.HelloWorld");
-    Object o2 = clazz2.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o2.toString());
-
-    // This is false because even though it's the same class, it's loaded from a different jar
-    // this is a change in behavior from previous versions of vfs2 where it would load the same
-    // class from different jars as though it was from the first jar
-    assertNotEquals(clazz1, clazz2);
-    assertNotSame(o1, o2);
-    assertEquals(clazz1.getName(), clazz2.getName());
-    assertEquals(o1.toString(), o2.toString());
-
-    arvcl.close();
-  }
-
-}
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java
deleted file mode 100644
index f850da7..0000000
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoaderTest.java
+++ /dev/null
@@ -1,210 +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.start.classloader.vfs;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.net.URLClassLoader;
-
-import org.apache.accumulo.start.classloader.AccumuloClassLoader;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor;
-import org.powermock.modules.junit4.PowerMockRunner;
-import org.powermock.reflect.Whitebox;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@SuppressWarnings("deprecation")
-@Deprecated
-@SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(AccumuloVFSClassLoader.class)
-@SuppressStaticInitializationFor({"org.apache.accumulo.start.classloader.AccumuloVFSClassLoader",
-    "org.apache.log4j.LogManager"})
-@PowerMockIgnore({"org.apache.log4j.*", "org.apache.hadoop.log.metrics",
-    "org.apache.commons.logging.*", "org.xml.*", "javax.xml.*", "org.w3c.dom.*",
-    "org.apache.hadoop.*", "com.sun.org.apache.xerces.*"})
-public class AccumuloVFSClassLoaderTest {
-
-  @Rule
-  public TemporaryFolder folder1 =
-      new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  /*
-   * Test that the default (empty dynamic class paths) does not create the 2nd level loader
-   */
-  @Test
-  public void testDefaultConfig() throws Exception {
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-
-    File conf = folder1.newFile("accumulo.properties");
-    FileWriter out = new FileWriter(conf, UTF_8);
-    out.append("general.classpaths=\n");
-    out.append("general.vfs.classpaths=\n");
-    out.close();
-
-    Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
-    ClassLoader acl = AccumuloVFSClassLoader.getClassLoader();
-    assertTrue((acl instanceof URLClassLoader));
-    // We can't check to see if the parent is an instance of BuiltinClassLoader
-    // Let's assert it's not something we now about
-    assertFalse((acl.getParent() instanceof VFSClassLoader));
-    assertFalse((acl.getParent() instanceof URLClassLoader));
-  }
-
-  /*
-   * Test that if configured with dynamic class paths, that the code creates the 2nd level loader
-   */
-  @Test
-  public void testDynamicConfig() throws Exception {
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-
-    File conf = folder1.newFile("accumulo.properties");
-    FileWriter out = new FileWriter(conf, UTF_8);
-    out.append("general.classpaths=\n");
-    out.append("general.vfs.classpaths=\n");
-    out.append("general.dynamic.classpaths=" + System.getProperty("user.dir") + "\n");
-    out.close();
-
-    Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
-    ClassLoader acl = AccumuloVFSClassLoader.getClassLoader();
-    assertTrue((acl instanceof VFSClassLoader));
-    assertTrue((acl.getParent() instanceof URLClassLoader));
-  }
-
-  /*
-   * Test with default context configured
-   */
-  @Test
-  public void testDefaultContextConfigured() throws Exception {
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-
-    // Copy jar file to TEST_DIR
-    FileUtils.copyURLToFile(this.getClass().getResource("/HelloWorld.jar"),
-        folder1.newFile("HelloWorld.jar"));
-
-    File conf = folder1.newFile("accumulo.properties");
-    FileWriter out = new FileWriter(conf, UTF_8);
-    out.append("general.classpaths=\n");
-    out.append(
-        "general.vfs.classpaths=" + new File(folder1.getRoot(), "HelloWorld.jar").toURI() + "\n");
-    out.append("general.dynamic.classpaths=" + System.getProperty("user.dir") + "\n");
-    out.close();
-
-    Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
-    ClassLoader acl = AccumuloVFSClassLoader.getClassLoader();
-    assertTrue((acl instanceof VFSClassLoader));
-    assertTrue((acl.getParent() instanceof VFSClassLoader));
-    VFSClassLoader arvcl = (VFSClassLoader) acl.getParent();
-    assertEquals(1, arvcl.getFileObjects().length);
-    // We can't be sure what the authority/host will be due to FQDN mappings, so just check the path
-    assertTrue(arvcl.getFileObjects()[0].getURL().toString().contains("HelloWorld.jar"));
-    Class<?> clazz1 = arvcl.loadClass("test.HelloWorld");
-    Object o1 = clazz1.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o1.toString());
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-  }
-
-  @Test
-  public void testDefaultCacheDirectory() throws Exception {
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-
-    File conf = folder1.newFile("accumulo.properties");
-    FileWriter out = new FileWriter(conf, UTF_8);
-    out.append("general.classpaths=\n");
-    out.append("general.vfs.classpaths=\n");
-    out.close();
-
-    Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
-    AccumuloVFSClassLoader.getClassLoader();
-    FileSystemManager manager = AccumuloVFSClassLoader.generateVfs();
-    UniqueFileReplicator replicator = Whitebox.getInternalState(manager, "fileReplicator");
-    File tempDir = Whitebox.getInternalState(replicator, "tempDir");
-    String tempDirParent = tempDir.getParent();
-    String tempDirName = tempDir.getName();
-    String javaIoTmpDir = System.getProperty("java.io.tmpdir");
-
-    // trim off any final separator, because java.io.File does the same.
-    if (javaIoTmpDir.endsWith(File.separator)) {
-      javaIoTmpDir = javaIoTmpDir.substring(0, javaIoTmpDir.length() - File.separator.length());
-    }
-
-    assertEquals(javaIoTmpDir, tempDirParent);
-    assertTrue(tempDirName.startsWith("accumulo-vfs-cache-"));
-    assertTrue(tempDirName.endsWith(System.getProperty("user.name", "nouser")));
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-  }
-
-  @Test
-  public void testCacheDirectoryConfigured() throws Exception {
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-    String cacheDir = "/some/random/cache/dir";
-
-    File conf = folder1.newFile("accumulo.properties");
-    FileWriter out = new FileWriter(conf, UTF_8);
-    out.append("general.classpaths=\n");
-    out.append(AccumuloVFSClassLoader.VFS_CACHE_DIR + "=" + cacheDir + "\n");
-    out.close();
-
-    Whitebox.setInternalState(AccumuloClassLoader.class, "accumuloConfigUrl", conf.toURI().toURL());
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "lock", new Object());
-    AccumuloVFSClassLoader.getClassLoader();
-    FileSystemManager manager = AccumuloVFSClassLoader.generateVfs();
-    UniqueFileReplicator replicator = Whitebox.getInternalState(manager, "fileReplicator");
-    File tempDir = Whitebox.getInternalState(replicator, "tempDir");
-    String tempDirParent = tempDir.getParent();
-    String tempDirName = tempDir.getName();
-    assertEquals(cacheDir, tempDirParent);
-    assertTrue(tempDirName.startsWith("accumulo-vfs-cache-"));
-    assertTrue(tempDirName.endsWith(System.getProperty("user.name", "nouser")));
-
-    Whitebox.setInternalState(AccumuloVFSClassLoader.class, "loader",
-        (AccumuloReloadingVFSClassLoader) null);
-  }
-}
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/ContextManagerTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/ContextManagerTest.java
deleted file mode 100644
index e77b74b..0000000
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/ContextManagerTest.java
+++ /dev/null
@@ -1,151 +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.start.classloader.vfs;
-
-import static org.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertNotSame;
-import static org.junit.jupiter.api.Assertions.assertSame;
-
-import java.io.File;
-import java.util.HashSet;
-import java.util.Objects;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.FileSystemManager;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@Deprecated
-@SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-public class ContextManagerTest {
-  @TempDir
-  private static File tempFolder;
-
-  private FileSystemManager vfs;
-  private File folder1;
-  private File folder2;
-  private String uri1;
-  private String uri2;
-
-  static FileSystemManager getVFS() {
-    try {
-      return AccumuloVFSClassLoader.generateVfs();
-    } catch (FileSystemException e) {
-      throw new RuntimeException("Error setting up VFS", e);
-    }
-  }
-
-  @BeforeEach
-  public void setup() throws Exception {
-    vfs = getVFS();
-    folder1 = new File(tempFolder, "folder1");
-    folder2 = new File(tempFolder, "folder2");
-
-    FileUtils.copyURLToFile(Objects.requireNonNull(this.getClass().getResource("/HelloWorld.jar")),
-        new File(folder1, "HelloWorld.jar"));
-    FileUtils.copyURLToFile(Objects.requireNonNull(this.getClass().getResource("/HelloWorld.jar")),
-        new File(folder2, "HelloWorld.jar"));
-
-    uri1 = new File(folder1, "HelloWorld.jar").toURI().toString();
-    uri2 = folder2.toURI() + ".*";
-
-  }
-
-  FileObject[] createFileSystems(FileObject[] fos) throws FileSystemException {
-    FileObject[] rfos = new FileObject[fos.length];
-    for (int i = 0; i < fos.length; i++) {
-      if (vfs.canCreateFileSystem(fos[i])) {
-        rfos[i] = vfs.createFileSystem(fos[i]);
-      } else {
-        rfos[i] = fos[i];
-      }
-    }
-
-    return rfos;
-  }
-
-  @Test
-  public void differentContexts() throws Exception {
-
-    ContextManager cm = new ContextManager(vfs, ClassLoader::getSystemClassLoader);
-
-    cm.setContextConfig(context -> {
-      if (context.equals("CX1")) {
-        return new ContextManager.ContextConfig("CX1", uri1, true);
-      } else if (context.equals("CX2")) {
-        return new ContextManager.ContextConfig("CX2", uri2, true);
-      }
-      return null;
-    });
-
-    FileObject testDir = vfs.resolveFile(folder1.toURI().toString());
-    FileObject[] dirContents = testDir.getChildren();
-    ClassLoader cl1 = cm.getClassLoader("CX1");
-    FileObject[] files = ((VFSClassLoader) cl1).getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents), files);
-
-    FileObject testDir2 = vfs.resolveFile(folder2.toURI().toString());
-    FileObject[] dirContents2 = testDir2.getChildren();
-    ClassLoader cl2 = cm.getClassLoader("CX2");
-    FileObject[] files2 = ((VFSClassLoader) cl2).getFileObjects();
-    assertArrayEquals(createFileSystems(dirContents2), files2);
-
-    Class<?> defaultContextClass = cl1.loadClass("test.HelloWorld");
-    Object o1 = defaultContextClass.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o1.toString());
-
-    Class<?> myContextClass = cl2.loadClass("test.HelloWorld");
-    Object o2 = myContextClass.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o2.toString());
-
-    assertNotEquals(defaultContextClass, myContextClass);
-
-    cm.removeUnusedContexts(new HashSet<>());
-  }
-
-  @Test
-  public void testPostDelegation() throws Exception {
-    final VFSClassLoader parent = new VFSClassLoader(new FileObject[] {vfs.resolveFile(uri1)}, vfs);
-
-    Class<?> pclass = parent.loadClass("test.HelloWorld");
-
-    ContextManager cm = new ContextManager(vfs, () -> parent);
-
-    cm.setContextConfig(context -> {
-      if (context.equals("CX1")) {
-        return new ContextManager.ContextConfig("CX1", uri1.toString(), true);
-      } else if (context.equals("CX2")) {
-        return new ContextManager.ContextConfig("CX2", uri2.toString(), false);
-      }
-      return null;
-    });
-
-    assertSame(cm.getClassLoader("CX1").loadClass("test.HelloWorld"), pclass);
-    assertNotSame(cm.getClassLoader("CX2").loadClass("test.HelloWorld"), pclass);
-  }
-
-}
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/WithTestNames.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/WithTestNames.java
deleted file mode 100644
index fea6f3b..0000000
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/WithTestNames.java
+++ /dev/null
@@ -1,42 +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.start.classloader.vfs;
-
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.TestInfo;
-
-// This is only for the unit tests and integration tests in this module
-// It must be copied for use in other modules, because tests in one module
-// don't have dependencies on other modules, and we can't put this in a
-// regular, non-test jar, because we don't want to add a dependency on
-// JUnit in a non-test jar
-public class WithTestNames {
-
-  private String testName;
-
-  @BeforeEach
-  public void setTestName(TestInfo info) {
-    testName = info.getTestMethod().orElseThrow().getName();
-  }
-
-  protected String testName() {
-    return testName;
-  }
-
-}
diff --git a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/providers/VfsClassLoaderTest.java b/start/src/test/java/org/apache/accumulo/start/classloader/vfs/providers/VfsClassLoaderTest.java
deleted file mode 100644
index 1f32b71..0000000
--- a/start/src/test/java/org/apache/accumulo/start/classloader/vfs/providers/VfsClassLoaderTest.java
+++ /dev/null
@@ -1,151 +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.start.classloader.vfs.providers;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.net.URL;
-
-import org.apache.accumulo.start.test.AccumuloDFSBase;
-import org.apache.commons.vfs2.FileChangeEvent;
-import org.apache.commons.vfs2.FileListener;
-import org.apache.commons.vfs2.FileObject;
-import org.apache.commons.vfs2.impl.DefaultFileMonitor;
-import org.apache.commons.vfs2.impl.VFSClassLoader;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-public class VfsClassLoaderTest extends AccumuloDFSBase {
-
-  private static final Path TEST_DIR = new Path(getHdfsUri() + "/test-dir");
-
-  private FileSystem hdfs = null;
-  private VFSClassLoader cl = null;
-
-  @BeforeEach
-  public void setup() throws Exception {
-
-    this.hdfs = cluster.getFileSystem();
-    this.hdfs.mkdirs(TEST_DIR);
-
-    // Copy jar file to TEST_DIR
-    URL jarPath = this.getClass().getResource("/HelloWorld.jar");
-    Path src = new Path(jarPath.toURI().toString());
-    Path dst = new Path(TEST_DIR, src.getName());
-    this.hdfs.copyFromLocalFile(src, dst);
-
-    FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString());
-    FileObject[] dirContents = testDir.getChildren();
-
-    // Point the VFSClassLoader to all of the objects in TEST_DIR
-    this.cl = new VFSClassLoader(dirContents, vfs);
-  }
-
-  @Test
-  public void testGetClass() throws Exception {
-    Class<?> helloWorldClass = this.cl.loadClass("test.HelloWorld");
-    Object o = helloWorldClass.getDeclaredConstructor().newInstance();
-    assertEquals("Hello World!", o.toString());
-  }
-
-  @Test
-  public void testFileMonitor() throws Exception {
-    MyFileMonitor listener = new MyFileMonitor();
-    DefaultFileMonitor monitor = new DefaultFileMonitor(listener);
-    monitor.setRecursive(true);
-    FileObject testDir = vfs.resolveFile(TEST_DIR.toUri().toString());
-    monitor.addFile(testDir);
-    monitor.start();
-
-    // Copy jar file to a new file name
-    URL jarPath = this.getClass().getResource("/HelloWorld.jar");
-    Path src = new Path(jarPath.toURI().toString());
-    Path dst = new Path(TEST_DIR, "HelloWorld2.jar");
-    this.hdfs.copyFromLocalFile(src, dst);
-
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-    assertTrue(listener.isFileCreated());
-
-    // Update the jar
-    jarPath = this.getClass().getResource("/HelloWorld.jar");
-    src = new Path(jarPath.toURI().toString());
-    dst = new Path(TEST_DIR, "HelloWorld2.jar");
-    this.hdfs.copyFromLocalFile(src, dst);
-
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-    assertTrue(listener.isFileChanged());
-
-    this.hdfs.delete(dst, false);
-    // VFS-487 significantly wait to avoid failure
-    Thread.sleep(7000);
-    assertTrue(listener.isFileDeleted());
-
-    monitor.stop();
-
-  }
-
-  @AfterEach
-  public void tearDown() throws Exception {
-    this.hdfs.delete(TEST_DIR, true);
-  }
-
-  public static class MyFileMonitor implements FileListener {
-
-    private boolean fileChanged = false;
-    private boolean fileDeleted = false;
-    private boolean fileCreated = false;
-
-    @Override
-    public void fileCreated(FileChangeEvent event) throws Exception {
-      // System.out.println(event.getFile() + " created");
-      this.fileCreated = true;
-    }
-
-    @Override
-    public void fileDeleted(FileChangeEvent event) throws Exception {
-      // System.out.println(event.getFile() + " deleted");
-      this.fileDeleted = true;
-    }
-
-    @Override
-    public void fileChanged(FileChangeEvent event) throws Exception {
-      // System.out.println(event.getFile() + " changed");
-      this.fileChanged = true;
-    }
-
-    public boolean isFileChanged() {
-      return fileChanged;
-    }
-
-    public boolean isFileDeleted() {
-      return fileDeleted;
-    }
-
-    public boolean isFileCreated() {
-      return fileCreated;
-    }
-
-  }
-}
diff --git a/start/src/test/java/org/apache/accumulo/start/test/AccumuloDFSBase.java b/start/src/test/java/org/apache/accumulo/start/test/AccumuloDFSBase.java
deleted file mode 100644
index dd0b444..0000000
--- a/start/src/test/java/org/apache/accumulo/start/test/AccumuloDFSBase.java
+++ /dev/null
@@ -1,138 +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.start.test;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-
-import org.apache.accumulo.start.classloader.vfs.MiniDFSUtil;
-import org.apache.commons.vfs2.CacheStrategy;
-import org.apache.commons.vfs2.FileSystemException;
-import org.apache.commons.vfs2.cache.DefaultFilesCache;
-import org.apache.commons.vfs2.cache.SoftRefFilesCache;
-import org.apache.commons.vfs2.impl.DefaultFileReplicator;
-import org.apache.commons.vfs2.impl.DefaultFileSystemManager;
-import org.apache.commons.vfs2.impl.FileContentInfoFilenameFactory;
-import org.apache.commons.vfs2.provider.hdfs.HdfsFileProvider;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-public class AccumuloDFSBase {
-
-  protected static Configuration conf = null;
-  protected static DefaultFileSystemManager vfs = null;
-  protected static MiniDFSCluster cluster = null;
-
-  private static URI HDFS_URI;
-
-  protected static URI getHdfsUri() {
-    return HDFS_URI;
-  }
-
-  @BeforeAll
-  public static void miniDfsClusterSetup() {
-    System.setProperty("java.io.tmpdir", System.getProperty("user.dir") + "/target");
-    // System.setProperty("org.apache.commons.logging.Log",
-    // "org.apache.commons.logging.impl.NoOpLog");
-    // Logger.getRootLogger().setLevel(Level.ERROR);
-
-    // Put the MiniDFSCluster directory in the target directory
-    System.setProperty("test.build.data", "target/build/test/data");
-
-    // Setup HDFS
-    conf = new Configuration();
-    conf.set("hadoop.security.token.service.use_ip", "true");
-
-    conf.set("dfs.datanode.data.dir.perm", MiniDFSUtil.computeDatanodeDirectoryPermission());
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024 * 1024); // 1M blocksize
-
-    try {
-      cluster = new MiniDFSCluster.Builder(conf).build();
-      cluster.waitClusterUp();
-      // We can't assume that the hostname of "localhost" will still be "localhost" after
-      // starting up the NameNode. We may get mapped into a FQDN via settings in /etc/hosts.
-      HDFS_URI = cluster.getFileSystem().getUri();
-    } catch (IOException e) {
-      throw new RuntimeException("Error setting up mini cluster", e);
-    }
-
-    // Set up the VFS
-    vfs = new DefaultFileSystemManager();
-    try {
-      vfs.setFilesCache(new DefaultFilesCache());
-      vfs.addProvider("res", new org.apache.commons.vfs2.provider.res.ResourceFileProvider());
-      vfs.addProvider("zip", new org.apache.commons.vfs2.provider.zip.ZipFileProvider());
-      vfs.addProvider("gz", new org.apache.commons.vfs2.provider.gzip.GzipFileProvider());
-      vfs.addProvider("ram", new org.apache.commons.vfs2.provider.ram.RamFileProvider());
-      vfs.addProvider("file",
-          new org.apache.commons.vfs2.provider.local.DefaultLocalFileProvider());
-      vfs.addProvider("jar", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-      vfs.addProvider("http", new org.apache.commons.vfs2.provider.http.HttpFileProvider());
-      vfs.addProvider("https", new org.apache.commons.vfs2.provider.https.HttpsFileProvider());
-      vfs.addProvider("ftp", new org.apache.commons.vfs2.provider.ftp.FtpFileProvider());
-      vfs.addProvider("ftps", new org.apache.commons.vfs2.provider.ftps.FtpsFileProvider());
-      vfs.addProvider("war", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-      vfs.addProvider("par", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-      vfs.addProvider("ear", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-      vfs.addProvider("sar", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-      vfs.addProvider("ejb3", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
-      vfs.addProvider("tmp", new org.apache.commons.vfs2.provider.temp.TemporaryFileProvider());
-      vfs.addProvider("tar", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
-      vfs.addProvider("tbz2", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
-      vfs.addProvider("tgz", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
-      vfs.addProvider("bz2", new org.apache.commons.vfs2.provider.bzip2.Bzip2FileProvider());
-      vfs.addProvider("hdfs", new HdfsFileProvider());
-      vfs.addExtensionMap("jar", "jar");
-      vfs.addExtensionMap("zip", "zip");
-      vfs.addExtensionMap("gz", "gz");
-      vfs.addExtensionMap("tar", "tar");
-      vfs.addExtensionMap("tbz2", "tar");
-      vfs.addExtensionMap("tgz", "tar");
-      vfs.addExtensionMap("bz2", "bz2");
-      vfs.addMimeTypeMap("application/java-archive", "jar");
-      vfs.addMimeTypeMap("application/x-tar", "tar");
-      vfs.addMimeTypeMap("application/x-gzip", "gz");
-      vfs.addMimeTypeMap("application/zip", "zip");
-      vfs.setFileContentInfoFactory(new FileContentInfoFilenameFactory());
-      vfs.setFilesCache(new SoftRefFilesCache());
-      vfs.setReplicator(new DefaultFileReplicator(new File(System.getProperty("java.io.tmpdir"),
-          "accumulo-vfs-cache-" + System.getProperty("user.name", "nouser"))));
-      vfs.setCacheStrategy(CacheStrategy.ON_RESOLVE);
-      vfs.init();
-    } catch (FileSystemException e) {
-      throw new RuntimeException("Error setting up VFS", e);
-    }
-
-  }
-
-  @AfterAll
-  public static void tearDownMiniDfsCluster() {
-    if (null != cluster) {
-      cluster.shutdown();
-    }
-  }
-
-}
diff --git a/start/src/test/resources/log4j2-test.properties b/start/src/test/resources/log4j2-test.properties
index 36951f8..da6150d 100644
--- a/start/src/test/resources/log4j2-test.properties
+++ b/start/src/test/resources/log4j2-test.properties
@@ -27,9 +27,5 @@
 appender.console.layout.type = PatternLayout
 appender.console.layout.pattern = [%t} %-5p %c %x - %m%n
 
-logger.01.name = org.apache.commons.vfs2.impl.DefaultFileSystemManager
-logger.01.level = error
-
 rootLogger.level = info
 rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/test/pom.xml b/test/pom.xml
index 07fb24f..7db2364 100644
--- a/test/pom.xml
+++ b/test/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-test</artifactId>
   <name>Apache Accumulo Testing</name>
@@ -55,10 +55,6 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-cli</groupId>
       <artifactId>commons-cli</artifactId>
     </dependency>
@@ -207,6 +203,10 @@
       <artifactId>junit-jupiter-engine</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
@@ -236,6 +236,23 @@
             <mapping>
               <shellit>SCRIPT_STYLE</shellit>
             </mapping>
+            <licenseSets>
+              <licenseSet>
+                <header>${rootlocation}/src/build/license-header.txt</header>
+                <excludes>
+                  <exclude>src/main/resources/v2_import_test/data/distcp.txt</exclude>
+                </excludes>
+              </licenseSet>
+            </licenseSets>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.rat</groupId>
+          <artifactId>apache-rat-plugin</artifactId>
+          <configuration>
+            <excludes>
+              <exclude>src/main/resources/v2_import_test/data/distcp.txt</exclude>
+            </excludes>
           </configuration>
         </plugin>
         <plugin>
diff --git a/test/src/main/java/org/apache/accumulo/harness/AccumuloITBase.java b/test/src/main/java/org/apache/accumulo/harness/AccumuloITBase.java
index 0c29ca8..f23a051 100644
--- a/test/src/main/java/org/apache/accumulo/harness/AccumuloITBase.java
+++ b/test/src/main/java/org/apache/accumulo/harness/AccumuloITBase.java
@@ -24,7 +24,6 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.Collection;
 import java.util.Map.Entry;
@@ -44,7 +43,6 @@
  * Methods, setup and/or infrastructure which are common to any Accumulo integration test.
  */
 public class AccumuloITBase extends WithTestNames {
-  public static final SecureRandom random = new SecureRandom();
   private static final Logger log = LoggerFactory.getLogger(AccumuloITBase.class);
 
   public static final String STANDALONE_CAPABLE_CLUSTER = "StandaloneCapableCluster";
@@ -133,7 +131,7 @@
   });
 
   /**
-   * Time to wait per-method before declaring a timeout, in seconds.
+   * Time to wait per-method before declaring a timeout.
    */
   protected Duration defaultTimeout() {
     return Duration.ofMinutes(10);
diff --git a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
index 2276318..a21c480 100644
--- a/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
+++ b/test/src/main/java/org/apache/accumulo/harness/SharedMiniClusterBase.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.harness;
 
 import static java.lang.StackWalker.Option.RETAIN_CLASS_REFERENCE;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -124,7 +125,7 @@
         StackWalker.getInstance(RETAIN_CLASS_REFERENCE).walk(findCallerITClass).map(Class::getName);
     // use the calling class name, or default to a unique name if IT class can't be found
     return callerClassName.orElse(String.format("UnknownITClass-%d-%d", System.currentTimeMillis(),
-        random.nextInt(Short.MAX_VALUE)));
+        RANDOM.get().nextInt(Short.MAX_VALUE)));
   }
 
   /**
diff --git a/test/src/main/java/org/apache/accumulo/test/AmpleIT.java b/test/src/main/java/org/apache/accumulo/test/AmpleIT.java
new file mode 100644
index 0000000..e89a1fd
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/AmpleIT.java
@@ -0,0 +1,106 @@
+/*
+ * 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.test;
+
+import static java.util.stream.Collectors.toSet;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Sets;
+
+public class AmpleIT extends AccumuloClusterHarness {
+
+  private void runFetchTest(Ample ample, List<KeyExtent> extentsToFetch, Set<KeyExtent> expected,
+      Set<KeyExtent> expectMissing) {
+    Set<KeyExtent> extentsSeen;
+    // always run a test without a consumer for not seen tablets as this takes a different code path
+    try (TabletsMetadata tm =
+        ample.readTablets().forTablets(extentsToFetch, Optional.empty()).build()) {
+      extentsSeen = tm.stream().map(TabletMetadata::getExtent).collect(toSet());
+      assertEquals(expected, extentsSeen);
+    }
+
+    HashSet<KeyExtent> extentsNotSeen = new HashSet<>();
+    try (TabletsMetadata tm =
+        ample.readTablets().forTablets(extentsToFetch, Optional.of(extentsNotSeen::add)).build()) {
+      extentsSeen = tm.stream().map(TabletMetadata::getExtent).collect(toSet());
+    }
+    assertEquals(expected, extentsSeen);
+    assertEquals(expectMissing, extentsNotSeen);
+  }
+
+  @Test
+  public void testFetchMultipleExtents() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      SortedSet<Text> splits = new TreeSet<>(List.of(new Text("c"), new Text("f"), new Text("v")));
+      NewTableConfiguration ntc = new NewTableConfiguration().withSplits(splits);
+      c.tableOperations().create(table, ntc);
+
+      var tableId = TableId.of(c.tableOperations().tableIdMap().get(table));
+
+      // extents that exist in the metadata table
+      KeyExtent ke1 = new KeyExtent(tableId, new Text("c"), null);
+      KeyExtent ke2 = new KeyExtent(tableId, new Text("f"), new Text("c"));
+      KeyExtent ke3 = new KeyExtent(tableId, new Text("v"), new Text("f"));
+      KeyExtent ke4 = new KeyExtent(tableId, null, new Text("v"));
+
+      // extents that do not exist in the metadata table
+      KeyExtent ne1 = new KeyExtent(tableId, null, new Text("g"));
+      KeyExtent ne2 = new KeyExtent(tableId, new Text("e"), new Text("c"));
+      KeyExtent ne3 = new KeyExtent(tableId, new Text("b"), null);
+      KeyExtent ne4 = new KeyExtent(TableId.of(tableId.canonical() + "not"), new Text("c"), null);
+
+      var ample = getServerContext().getAmple();
+
+      var toFetch = new ArrayList<KeyExtent>();
+
+      for (var existing : Sets.powerSet(Set.of(ke1, ke2, ke3, ke4))) {
+        for (var nonexisting : Sets.powerSet(Set.of(ne1, ne2, ne3, ne4))) {
+          toFetch.clear();
+          toFetch.addAll(existing);
+          toFetch.addAll(nonexisting);
+
+          // run test to ensure when ample fetches multiple extents it handles one that do exist in
+          // the metadata table and those that do not
+          runFetchTest(ample, toFetch, existing, nonexisting);
+        }
+      }
+    }
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index 14beb2b..9239674 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -18,16 +18,16 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.fail;
 
 import java.time.Duration;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
@@ -38,10 +38,11 @@
 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.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -100,14 +101,14 @@
       zcache.clear();
       var path = ServiceLock
           .path(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZGC_LOCK);
-      byte[] gcLockData;
+      Optional<ServiceLockData> gcLockData;
       do {
         gcLockData = ServiceLock.getLockData(zcache, path, null);
-        if (gcLockData != null) {
+        if (gcLockData.isPresent()) {
           log.info("Waiting for GC ZooKeeper lock to expire");
           Thread.sleep(2000);
         }
-      } while (gcLockData != null);
+      } while (gcLockData.isPresent());
 
       log.info("GC lock was lost");
 
@@ -116,11 +117,11 @@
 
       do {
         gcLockData = ServiceLock.getLockData(zcache, path, null);
-        if (gcLockData == null) {
+        if (gcLockData.isEmpty()) {
           log.info("Waiting for GC ZooKeeper lock to be acquired");
           Thread.sleep(2000);
         }
-      } while (gcLockData == null);
+      } while (gcLockData.isEmpty());
 
       log.info("GC lock was acquired");
     }
@@ -165,10 +166,11 @@
       c.tableOperations().delete(tableName);
       log.info("Sleeping to let garbage collector run");
       // let gc run
-      sleepUninterruptibly(timeoutFactor * 15, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(timeoutFactor * 15L));
       log.info("Verifying that delete markers were deleted");
       // look for delete markers
-      try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.setRange(DeletesSection.getRange());
         for (Entry<Key,Value> entry : scanner) {
           String row = entry.getKey().getRow().toString();
diff --git a/test/src/main/java/org/apache/accumulo/test/BatchWriterIT.java b/test/src/main/java/org/apache/accumulo/test/BatchWriterIT.java
index 57c3ec9..6934277 100644
--- a/test/src/main/java/org/apache/accumulo/test/BatchWriterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BatchWriterIT.java
@@ -18,40 +18,15 @@
  */
 package org.apache.accumulo.test;
 
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
 import java.time.Duration;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-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.admin.NewTableConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.conf.Property;
-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.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.rpc.ThriftUtil;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
-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.harness.AccumuloClusterHarness;
-import org.apache.accumulo.test.constraints.NumericValueConstraint;
-import org.apache.hadoop.io.Text;
-import org.apache.thrift.TServiceClient;
 import org.junit.jupiter.api.Test;
 
 public class BatchWriterIT extends AccumuloClusterHarness {
@@ -76,74 +51,4 @@
       }
     }
   }
-
-  private static void update(ClientContext context, Mutation m, KeyExtent extent) throws Exception {
-
-    TabletLocator.TabletLocation tabLoc = TabletLocator.getLocator(context, extent.tableId())
-        .locateTablet(context, new Text(m.getRow()), false, true);
-
-    var server = HostAndPort.fromString(tabLoc.tablet_location);
-
-    TabletClientService.Iface client = null;
-    try {
-      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, server, context);
-      client.update(TraceUtil.traceInfo(), context.rpcCreds(), extent.toThrift(), m.toThrift(),
-          TDurability.DEFAULT);
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code);
-    } finally {
-      ThriftUtil.returnClient((TServiceClient) client, context);
-    }
-  }
-
-  static String toString(Map.Entry<Key,Value> e) {
-    return e.getKey().getRow() + ":" + e.getKey().getColumnFamily() + ":"
-        + e.getKey().getColumnQualifier() + ":" + e.getKey().getColumnVisibility() + ":"
-        + e.getValue();
-  }
-
-  @Test
-  public void testSingleMutationWriteRPC() throws Exception {
-    // The batchwriter used to use this RPC and no longer does. This test exist to exercise the
-    // unused RPC until its removed in 3.x. Older client versions of Accumulo 2.1.x may call this
-    // RPC.
-
-    String table = getUniqueNames(1)[0];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      NewTableConfiguration ntc = new NewTableConfiguration();
-      ntc.setProperties(Map.of(Property.TABLE_CONSTRAINT_PREFIX.getKey() + "1",
-          NumericValueConstraint.class.getName()));
-      c.tableOperations().create(table, ntc);
-
-      var tableId = TableId.of(c.tableOperations().tableIdMap().get(table));
-
-      Mutation m = new Mutation("r1");
-      m.put("f1", "q3", new Value("1"));
-      m.put("f1", "q4", new Value("2"));
-
-      update((ClientContext) c, m, new KeyExtent(tableId, null, null));
-
-      try (var scanner = c.createScanner(table)) {
-        var entries = scanner.stream().map(BatchWriterIT::toString).collect(Collectors.toList());
-        assertEquals(List.of("r1:f1:q3::1", "r1:f1:q4::2"), entries);
-      }
-
-      m = new Mutation("r1");
-      m.put("f1", "q3", new Value("5"));
-      m.put("f1", "q7", new Value("3"));
-
-      update((ClientContext) c, m, new KeyExtent(tableId, null, null));
-
-      try (var scanner = c.createScanner(table)) {
-        var entries = scanner.stream().map(BatchWriterIT::toString).collect(Collectors.toList());
-        assertEquals(List.of("r1:f1:q3::5", "r1:f1:q4::2", "r1:f1:q7::3"), entries);
-      }
-
-      var m2 = new Mutation("r2");
-      m2.put("f1", "q1", new Value("abc"));
-      assertThrows(ConstraintViolationException.class,
-          () -> update((ClientContext) c, m2, new KeyExtent(tableId, null, null)));
-    }
-
-  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java b/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
deleted file mode 100644
index f8d423f..0000000
--- a/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
+++ /dev/null
@@ -1,154 +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.test;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-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.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.rfile.RFile;
-import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-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.junit.jupiter.api.Test;
-
-public class BulkImportMonitoringIT extends ConfigurableMacBase {
-
-  @Override
-  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
-    cfg.useMiniDFS(true);
-  }
-
-  // suppress importDirectory deprecated since this tests legacy monitoring
-  @SuppressWarnings("deprecation")
-  @Test
-  public void test() throws Exception {
-    getCluster().getClusterControl().start(ServerType.MONITOR);
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-
-      // creating table name
-      final String tableName = getUniqueNames(1)[0];
-      // creating splits
-      SortedSet<Text> splits = new TreeSet<>();
-      for (int i = 1; i < 0xf; i++) {
-        splits.add(new Text(Integer.toHexString(i)));
-      }
-      // creating properties
-      HashMap<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_MAJC_RATIO.getKey(), "1");
-      // creating table with configuration
-      var ntc = new NewTableConfiguration().setProperties(props).withSplits(splits);
-      c.tableOperations().create(tableName, ntc);
-
-      ManagerMonitorInfo stats = getCluster().getManagerMonitorInfo();
-      assertEquals(1, stats.tServerInfo.size());
-      assertEquals(0, stats.bulkImports.size());
-      assertEquals(0, stats.tServerInfo.get(0).bulkImports.size());
-
-      log.info("Creating lots of bulk import files");
-      final FileSystem fs = getCluster().getFileSystem();
-      final Path basePath = getCluster().getTemporaryPath();
-
-      final Path base = new Path(basePath, "testBulkLoad" + tableName);
-      fs.delete(base, true);
-      fs.mkdirs(base);
-
-      ExecutorService es = Executors.newFixedThreadPool(5);
-      List<Future<Pair<String,String>>> futures = new ArrayList<>();
-      for (int i = 0; i < 10; i++) {
-        final int which = i;
-        futures.add(es.submit(() -> {
-          Path bulkFailures = new Path(base, "failures" + which);
-          Path files = new Path(base, "files" + which);
-          fs.mkdirs(bulkFailures);
-          fs.mkdirs(files);
-          for (int i1 = 0; i1 < 10; i1++) {
-            FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-                .forFile(files + "/bulk_" + i1 + "." + RFile.EXTENSION, fs, fs.getConf(),
-                    NoCryptoServiceFactory.NONE)
-                .withTableConfiguration(DefaultConfiguration.getInstance()).build();
-            writer.startDefaultLocalityGroup();
-            for (int j = 0x100; j < 0xfff; j += 3) {
-              writer.append(new Key(Integer.toHexString(j)), new Value());
-            }
-            writer.close();
-          }
-          return new Pair<>(files.toString(), bulkFailures.toString());
-        }));
-      }
-      List<Pair<String,String>> dirs = new ArrayList<>();
-      for (Future<Pair<String,String>> f : futures) {
-        dirs.add(f.get());
-      }
-      log.info("Importing");
-      long now = System.currentTimeMillis();
-      List<Future<Object>> errs = new ArrayList<>();
-      for (Pair<String,String> entry : dirs) {
-        final String dir = entry.getFirst();
-        final String err = entry.getSecond();
-        errs.add(es.submit(() -> {
-          c.tableOperations().importDirectory(tableName, dir, err, false);
-          return null;
-        }));
-      }
-      es.shutdown();
-      while (!es.isTerminated()
-          && stats.bulkImports.size() + stats.tServerInfo.get(0).bulkImports.size() == 0) {
-        es.awaitTermination(10, TimeUnit.MILLISECONDS);
-        stats = getCluster().getManagerMonitorInfo();
-      }
-      log.info(stats.bulkImports.toString());
-      assertTrue(!stats.bulkImports.isEmpty());
-      // look for exception
-      for (Future<Object> err : errs) {
-        err.get();
-      }
-      es.awaitTermination(2, TimeUnit.MINUTES);
-      assertTrue(es.isTerminated());
-      log.info(
-          String.format("Completed in %.2f seconds", (System.currentTimeMillis() - now) / 1000.));
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/BulkImportVolumeIT.java b/test/src/main/java/org/apache/accumulo/test/BulkImportVolumeIT.java
deleted file mode 100644
index 759b3f6..0000000
--- a/test/src/main/java/org/apache/accumulo/test/BulkImportVolumeIT.java
+++ /dev/null
@@ -1,104 +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.test;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.time.Duration;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-// ACCUMULO-118/ACCUMULO-2504
-public class BulkImportVolumeIT extends AccumuloClusterHarness {
-  private static final Logger log = LoggerFactory.getLogger(BulkImportVolumeIT.class);
-
-  File volDirBase = null;
-  Path v1, v2;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(1);
-  }
-
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    File baseDir = cfg.getDir();
-    volDirBase = new File(baseDir, "volumes");
-    File v1f = new File(volDirBase, "v1");
-    File v2f = new File(volDirBase, "v2");
-    v1 = new Path("file://" + v1f.getAbsolutePath());
-    v2 = new Path("file://" + v2f.getAbsolutePath());
-
-    // Run MAC on two locations in the local file system
-    cfg.setProperty(Property.INSTANCE_VOLUMES, v1 + "," + v2);
-
-    // use raw local file system so walogs sync and flush will work
-    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  // suppress importDirectory deprecated since this tests legacy failure directory
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testBulkImportFailure() throws Exception {
-    String tableName = getUniqueNames(1)[0];
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      client.tableOperations().create(tableName);
-      FileSystem fs = getFileSystem();
-      Path rootPath = new Path(cluster.getTemporaryPath(), getClass().getName());
-      fs.deleteOnExit(rootPath);
-
-      Path bulk = new Path(rootPath, "bulk");
-      fs.deleteOnExit(bulk);
-      log.info("bulk: {}", bulk);
-      if (fs.exists(bulk)) {
-        fs.delete(bulk, true);
-      }
-      assertTrue(fs.mkdirs(bulk));
-      Path err = new Path(rootPath, "err");
-      fs.deleteOnExit(err);
-      log.info("err: {}", err);
-      if (fs.exists(err)) {
-        fs.delete(err, true);
-      }
-      assertTrue(fs.mkdirs(err));
-      Path bogus = new Path(bulk, "bogus.rf");
-      fs.deleteOnExit(bogus);
-      fs.create(bogus).close();
-      log.info("bogus: {}", bogus);
-      assertTrue(fs.exists(bogus));
-      log.info("Importing {} into {} with failures directory {}", bulk, tableName, err);
-      client.tableOperations().importDirectory(tableName, bulk.toString(), err.toString(), false);
-      assertEquals(1, fs.listStatus(err).length);
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java b/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java
index fce97a4..73af744 100644
--- a/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java
+++ b/test/src/main/java/org/apache/accumulo/test/ChaoticLoadBalancer.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.harness.AccumuloITBase.random;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -30,7 +30,7 @@
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.TabletId;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.spi.balancer.BalancerEnvironment;
 import org.apache.accumulo.core.spi.balancer.TabletBalancer;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
@@ -85,7 +85,7 @@
     }
 
     for (TabletId tabletId : params.unassignedTablets().keySet()) {
-      int index = random.nextInt(tServerArray.size());
+      int index = RANDOM.get().nextInt(tServerArray.size());
       TabletServerId dest = tServerArray.get(index);
       params.addAssignment(tabletId, dest);
       long remaining = toAssign.get(dest) - 1;
@@ -115,7 +115,7 @@
     }
     problemReporter.clearProblemReportTimes();
 
-    boolean moveMetadata = random.nextInt(4) == 0;
+    boolean moveMetadata = RANDOM.get().nextInt(4) == 0;
     long totalTablets = 0;
     for (Entry<TabletServerId,TServerStatus> e : params.currentStatus().entrySet()) {
       long tabletCount = 0;
@@ -133,12 +133,12 @@
     for (Entry<TabletServerId,TServerStatus> e : params.currentStatus().entrySet()) {
       for (String tableId : e.getValue().getTableMap().keySet()) {
         TableId id = TableId.of(tableId);
-        if (!moveMetadata && MetadataTable.ID.equals(id)) {
+        if (!moveMetadata && AccumuloTable.METADATA.tableId().equals(id)) {
           continue;
         }
         try {
           for (TabletStatistics ts : getOnlineTabletsForTable(e.getKey(), id)) {
-            int index = random.nextInt(underCapacityTServer.size());
+            int index = RANDOM.get().nextInt(underCapacityTServer.size());
             TabletServerId dest = underCapacityTServer.get(index);
             if (dest.equals(e.getKey())) {
               continue;
diff --git a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
index d927acb..2a69f80 100644
--- a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java
@@ -18,12 +18,11 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -35,8 +34,7 @@
 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.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.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
@@ -84,12 +82,14 @@
 
       getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
 
-      for (String table : new String[] {MetadataTable.NAME, RootTable.NAME}) {
+      for (String table : new String[] {AccumuloTable.METADATA.tableName(),
+          AccumuloTable.ROOT.tableName()}) {
         client.tableOperations().flush(table, null, null, true);
       }
       log.debug("Checking entries for {}", tableName);
       assertEquals(1, count(tableName, client));
-      for (String table : new String[] {MetadataTable.NAME, RootTable.NAME}) {
+      for (String table : new String[] {AccumuloTable.METADATA.tableName(),
+          AccumuloTable.ROOT.tableName()}) {
         log.debug("Checking logs for {}", table);
         assertEquals(0, countLogs(client), "Found logs for " + table);
       }
@@ -101,11 +101,11 @@
       }
       assertEquals(0, count(tableName, client));
       client.tableOperations().flush(tableName, null, null, true);
-      client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-      client.tableOperations().flush(RootTable.NAME, null, null, true);
+      client.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+      client.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
       try {
         getCluster().getClusterControl().stopAllServers(ServerType.TABLET_SERVER);
-        sleepUninterruptibly(3, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(3));
       } finally {
         getCluster().getClusterControl().startAllServers(ServerType.TABLET_SERVER);
       }
@@ -115,7 +115,8 @@
 
   private int countLogs(AccumuloClient client) throws TableNotFoundException {
     int count = 0;
-    try (Scanner scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       scanner.fetchColumnFamily(LogColumnFamily.NAME);
       scanner.setRange(TabletsSection.getRange());
       for (Entry<Key,Value> entry : scanner) {
diff --git a/test/src/main/java/org/apache/accumulo/test/CloneIT.java b/test/src/main/java/org/apache/accumulo/test/CloneIT.java
index e2b4b9c..74dad97 100644
--- a/test/src/main/java/org/apache/accumulo/test/CloneIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CloneIT.java
@@ -24,6 +24,7 @@
 
 import java.util.HashSet;
 import java.util.Map.Entry;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -31,9 +32,11 @@
 import org.apache.accumulo.core.client.Scanner;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
@@ -42,8 +45,14 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.ArgumentsProvider;
+import org.junit.jupiter.params.provider.ArgumentsSource;
 
 public class CloneIT extends AccumuloClusterHarness {
 
@@ -74,8 +83,9 @@
     }
   }
 
-  @Test
-  public void testFilesChange() throws Exception {
+  @ParameterizedTest
+  @ArgumentsSource(RangeArgumentsProvider.class)
+  public void testFilesChange(Range range1, Range range2) throws Exception {
     String filePrefix = "hdfs://nn:8000/accumulo/tables/0";
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
@@ -86,7 +96,8 @@
 
       ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0"));
       ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value("/default_tablet"));
-      mut.put(DataFileColumnFamily.NAME.toString(), filePrefix + "/default_tablet/0_0.rf",
+      mut.put(DataFileColumnFamily.NAME.toString(),
+          getMetadata(filePrefix + "/default_tablet/0_0.rf", range1),
           new DataFileValue(1, 200).encodeAsString());
 
       try (BatchWriter bw1 = client.createBatchWriter(tableName);
@@ -98,8 +109,10 @@
         MetadataTableUtil.initializeClone(tableName, TableId.of("0"), TableId.of("1"), client, bw2);
 
         Mutation mut2 = new Mutation(ke.toMetaRow());
-        mut2.putDelete(DataFileColumnFamily.NAME.toString(), filePrefix + "/default_tablet/0_0.rf");
-        mut2.put(DataFileColumnFamily.NAME.toString(), filePrefix + "/default_tablet/1_0.rf",
+        mut2.putDelete(DataFileColumnFamily.NAME.toString(),
+            getMetadata(filePrefix + "/default_tablet/0_0.rf", range1));
+        mut2.put(DataFileColumnFamily.NAME.toString(),
+            getMetadata(filePrefix + "/default_tablet/1_0.rf", range2),
             new DataFileValue(2, 300).encodeAsString());
 
         bw1.addMutation(mut2);
@@ -126,13 +139,14 @@
         }
       }
       assertEquals(1, files.size());
-      assertTrue(files.contains(filePrefix + "/default_tablet/1_0.rf"));
+      assertTrue(files.contains(getMetadata(filePrefix + "/default_tablet/1_0.rf", range2)));
     }
   }
 
   // test split where files of children are the same
-  @Test
-  public void testSplit1() throws Exception {
+  @ParameterizedTest
+  @ArgumentsSource(RangeArgumentsProvider.class)
+  public void testSplit1(Range range) throws Exception {
     String filePrefix = "hdfs://nn:8000/accumulo/tables/0";
 
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
@@ -142,16 +156,16 @@
       try (BatchWriter bw1 = client.createBatchWriter(tableName);
           BatchWriter bw2 = client.createBatchWriter(tableName)) {
         bw1.addMutation(createTablet("0", null, null, "/default_tablet",
-            filePrefix + "/default_tablet/0_0.rf"));
+            filePrefix + "/default_tablet/0_0.rf", range));
 
         bw1.flush();
 
         MetadataTableUtil.initializeClone(tableName, TableId.of("0"), TableId.of("1"), client, bw2);
 
+        bw1.addMutation(createTablet("0", "m", null, "/default_tablet",
+            filePrefix + "/default_tablet/0_0.rf", range));
         bw1.addMutation(
-            createTablet("0", "m", null, "/default_tablet", filePrefix + "/default_tablet/0_0.rf"));
-        bw1.addMutation(
-            createTablet("0", null, "m", "/t-1", filePrefix + "/default_tablet/0_0.rf"));
+            createTablet("0", null, "m", "/t-1", filePrefix + "/default_tablet/0_0.rf", range));
 
         bw1.flush();
 
@@ -174,13 +188,14 @@
       }
       assertEquals(1, count);
       assertEquals(1, files.size());
-      assertTrue(files.contains(filePrefix + "/default_tablet/0_0.rf"));
+      assertTrue(files.contains(getMetadata(filePrefix + "/default_tablet/0_0.rf", range)));
     }
   }
 
   // test split where files of children differ... like majc and split occurred
-  @Test
-  public void testSplit2() throws Exception {
+  @ParameterizedTest
+  @ArgumentsSource(RangeArgumentsProvider.class)
+  public void testSplit2(Range range) throws Exception {
     String filePrefix = "hdfs://nn:8000/accumulo/tables/0";
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
@@ -189,16 +204,18 @@
       try (BatchWriter bw1 = client.createBatchWriter(tableName);
           BatchWriter bw2 = client.createBatchWriter(tableName)) {
         bw1.addMutation(createTablet("0", null, null, "/default_tablet",
-            filePrefix + "/default_tablet/0_0.rf"));
+            filePrefix + "/default_tablet/0_0.rf", range));
 
         bw1.flush();
 
         MetadataTableUtil.initializeClone(tableName, TableId.of("0"), TableId.of("1"), client, bw2);
 
-        bw1.addMutation(
-            createTablet("0", "m", null, "/default_tablet", filePrefix + "/default_tablet/1_0.rf"));
-        Mutation mut3 = createTablet("0", null, "m", "/t-1", filePrefix + "/default_tablet/1_0.rf");
-        mut3.putDelete(DataFileColumnFamily.NAME.toString(), filePrefix + "/default_tablet/0_0.rf");
+        bw1.addMutation(createTablet("0", "m", null, "/default_tablet",
+            filePrefix + "/default_tablet/1_0.rf", range));
+        Mutation mut3 =
+            createTablet("0", null, "m", "/t-1", filePrefix + "/default_tablet/1_0.rf", range);
+        mut3.putDelete(DataFileColumnFamily.NAME.toString(),
+            getMetadata(filePrefix + "/default_tablet/0_0.rf", range));
         bw1.addMutation(mut3);
 
         bw1.flush();
@@ -226,39 +243,41 @@
       }
       assertEquals(1, files.size());
       assertEquals(2, count);
-      assertTrue(files.contains(filePrefix + "/default_tablet/1_0.rf"));
+      assertTrue(files.contains(getMetadata(filePrefix + "/default_tablet/1_0.rf", range)));
     }
   }
 
-  private static Mutation deleteTablet(String tid, String endRow, String prevRow, String file) {
+  private static Mutation deleteTablet(String tid, String endRow, String prevRow, String file,
+      Range range) {
     KeyExtent ke = new KeyExtent(TableId.of(tid), endRow == null ? null : new Text(endRow),
         prevRow == null ? null : new Text(prevRow));
     Mutation mut = new Mutation(ke.toMetaRow());
     TabletColumnFamily.PREV_ROW_COLUMN.putDelete(mut);
     ServerColumnFamily.TIME_COLUMN.putDelete(mut);
     ServerColumnFamily.DIRECTORY_COLUMN.putDelete(mut);
-    mut.putDelete(DataFileColumnFamily.NAME.toString(), file);
+    mut.putDelete(DataFileColumnFamily.NAME.toString(), getMetadata(file, range));
 
     return mut;
   }
 
   private static Mutation createTablet(String tid, String endRow, String prevRow, String dir,
-      String file) {
+      String file, Range range) {
     KeyExtent ke = new KeyExtent(TableId.of(tid), endRow == null ? null : new Text(endRow),
         prevRow == null ? null : new Text(prevRow));
     Mutation mut = TabletColumnFamily.createPrevRowMutation(ke);
 
     ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0"));
     ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value(dir));
-    mut.put(DataFileColumnFamily.NAME.toString(), file,
+    mut.put(DataFileColumnFamily.NAME.toString(), getMetadata(file, range),
         new DataFileValue(10, 200).encodeAsString());
 
     return mut;
   }
 
   // test two tablets splitting into four
-  @Test
-  public void testSplit3() throws Exception {
+  @ParameterizedTest
+  @ArgumentsSource(RangeArgumentsProvider.class)
+  public void testSplit3(Range range1, Range range2, Range range3) throws Exception {
     String filePrefix = "hdfs://nn:8000/accumulo/tables/0";
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
@@ -266,17 +285,17 @@
 
       try (BatchWriter bw1 = client.createBatchWriter(tableName);
           BatchWriter bw2 = client.createBatchWriter(tableName)) {
-        bw1.addMutation(createTablet("0", "m", null, "/d1", filePrefix + "/d1/file1.rf"));
-        bw1.addMutation(createTablet("0", null, "m", "/d2", filePrefix + "/d2/file2.rf"));
+        bw1.addMutation(createTablet("0", "m", null, "/d1", filePrefix + "/d1/file1.rf", range1));
+        bw1.addMutation(createTablet("0", null, "m", "/d2", filePrefix + "/d2/file2.rf", range2));
 
         bw1.flush();
 
         MetadataTableUtil.initializeClone(tableName, TableId.of("0"), TableId.of("1"), client, bw2);
 
-        bw1.addMutation(createTablet("0", "f", null, "/d1", filePrefix + "/d1/file3.rf"));
-        bw1.addMutation(createTablet("0", "m", "f", "/d3", filePrefix + "/d1/file1.rf"));
-        bw1.addMutation(createTablet("0", "s", "m", "/d2", filePrefix + "/d2/file2.rf"));
-        bw1.addMutation(createTablet("0", null, "s", "/d4", filePrefix + "/d2/file2.rf"));
+        bw1.addMutation(createTablet("0", "f", null, "/d1", filePrefix + "/d1/file3.rf", range3));
+        bw1.addMutation(createTablet("0", "m", "f", "/d3", filePrefix + "/d1/file1.rf", range1));
+        bw1.addMutation(createTablet("0", "s", "m", "/d2", filePrefix + "/d2/file2.rf", range2));
+        bw1.addMutation(createTablet("0", null, "s", "/d4", filePrefix + "/d2/file2.rf", range2));
 
         bw1.flush();
 
@@ -300,14 +319,15 @@
       }
       assertEquals(2, count);
       assertEquals(2, files.size());
-      assertTrue(files.contains(filePrefix + "/d1/file1.rf"));
-      assertTrue(files.contains(filePrefix + "/d2/file2.rf"));
+      assertTrue(files.contains(getMetadata(filePrefix + "/d1/file1.rf", range1)));
+      assertTrue(files.contains(getMetadata(filePrefix + "/d2/file2.rf", range2)));
     }
   }
 
   // test cloned marker
-  @Test
-  public void testClonedMarker() throws Exception {
+  @ParameterizedTest
+  @ArgumentsSource(RangeArgumentsProvider.class)
+  public void testClonedMarker(Range range1, Range range2, Range range3) throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
       client.tableOperations().create(tableName);
@@ -315,22 +335,22 @@
 
       try (BatchWriter bw1 = client.createBatchWriter(tableName);
           BatchWriter bw2 = client.createBatchWriter(tableName)) {
-        bw1.addMutation(createTablet("0", "m", null, "/d1", filePrefix + "/d1/file1.rf"));
-        bw1.addMutation(createTablet("0", null, "m", "/d2", filePrefix + "/d2/file2.rf"));
+        bw1.addMutation(createTablet("0", "m", null, "/d1", filePrefix + "/d1/file1.rf", range1));
+        bw1.addMutation(createTablet("0", null, "m", "/d2", filePrefix + "/d2/file2.rf", range2));
 
         bw1.flush();
 
         MetadataTableUtil.initializeClone(tableName, TableId.of("0"), TableId.of("1"), client, bw2);
 
-        bw1.addMutation(deleteTablet("0", "m", null, filePrefix + "/d1/file1.rf"));
-        bw1.addMutation(deleteTablet("0", null, "m", filePrefix + "/d2/file2.rf"));
+        bw1.addMutation(deleteTablet("0", "m", null, filePrefix + "/d1/file1.rf", range1));
+        bw1.addMutation(deleteTablet("0", null, "m", filePrefix + "/d2/file2.rf", range2));
 
         bw1.flush();
 
-        bw1.addMutation(createTablet("0", "f", null, "/d1", filePrefix + "/d1/file3.rf"));
-        bw1.addMutation(createTablet("0", "m", "f", "/d3", filePrefix + "/d1/file1.rf"));
-        bw1.addMutation(createTablet("0", "s", "m", "/d2", filePrefix + "/d2/file3.rf"));
-        bw1.addMutation(createTablet("0", null, "s", "/d4", filePrefix + "/d4/file3.rf"));
+        bw1.addMutation(createTablet("0", "f", null, "/d1", filePrefix + "/d1/file3.rf", range3));
+        bw1.addMutation(createTablet("0", "m", "f", "/d3", filePrefix + "/d1/file1.rf", range1));
+        bw1.addMutation(createTablet("0", "s", "m", "/d2", filePrefix + "/d2/file3.rf", range3));
+        bw1.addMutation(createTablet("0", null, "s", "/d4", filePrefix + "/d4/file3.rf", range3));
 
         bw1.flush();
 
@@ -339,11 +359,11 @@
 
         assertEquals(1, rc);
 
-        bw1.addMutation(deleteTablet("0", "m", "f", filePrefix + "/d1/file1.rf"));
+        bw1.addMutation(deleteTablet("0", "m", "f", filePrefix + "/d1/file1.rf", range1));
 
         bw1.flush();
 
-        bw1.addMutation(createTablet("0", "m", "f", "/d3", filePrefix + "/d1/file3.rf"));
+        bw1.addMutation(createTablet("0", "m", "f", "/d3", filePrefix + "/d1/file3.rf", range3));
 
         bw1.flush();
 
@@ -365,15 +385,19 @@
       }
       assertEquals(3, count);
       assertEquals(3, files.size());
-      assertTrue(files.contains("hdfs://nn:8000/accumulo/tables/0/d1/file1.rf"));
-      assertTrue(files.contains("hdfs://nn:8000/accumulo/tables/0/d2/file3.rf"));
-      assertTrue(files.contains("hdfs://nn:8000/accumulo/tables/0/d4/file3.rf"));
+      assertTrue(
+          files.contains(getMetadata("hdfs://nn:8000/accumulo/tables/0/d1/file1.rf", range1)));
+      assertTrue(
+          files.contains(getMetadata("hdfs://nn:8000/accumulo/tables/0/d2/file3.rf", range3)));
+      assertTrue(
+          files.contains(getMetadata("hdfs://nn:8000/accumulo/tables/0/d4/file3.rf", range3)));
     }
   }
 
   // test two tablets splitting into four
-  @Test
-  public void testMerge() throws Exception {
+  @ParameterizedTest
+  @ArgumentsSource(RangeArgumentsProvider.class)
+  public void testMerge(Range range1, Range range2) throws Exception {
     String filePrefix = "hdfs://nn:8000/accumulo/tables/0";
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
@@ -381,16 +405,17 @@
 
       try (BatchWriter bw1 = client.createBatchWriter(tableName);
           BatchWriter bw2 = client.createBatchWriter(tableName)) {
-        bw1.addMutation(createTablet("0", "m", null, "/d1", filePrefix + "/d1/file1.rf"));
-        bw1.addMutation(createTablet("0", null, "m", "/d2", filePrefix + "/d2/file2.rf"));
+        bw1.addMutation(createTablet("0", "m", null, "/d1", filePrefix + "/d1/file1.rf", range1));
+        bw1.addMutation(createTablet("0", null, "m", "/d2", filePrefix + "/d2/file2.rf", range2));
 
         bw1.flush();
 
         MetadataTableUtil.initializeClone(tableName, TableId.of("0"), TableId.of("1"), client, bw2);
 
-        bw1.addMutation(deleteTablet("0", "m", null, filePrefix + "/d1/file1.rf"));
-        Mutation mut = createTablet("0", null, null, "/d2", filePrefix + "/d2/file2.rf");
-        mut.put(DataFileColumnFamily.NAME.toString(), filePrefix + "/d1/file1.rf",
+        bw1.addMutation(deleteTablet("0", "m", null, filePrefix + "/d1/file1.rf", range1));
+        Mutation mut = createTablet("0", null, null, "/d2", filePrefix + "/d2/file2.rf", range2);
+        mut.put(DataFileColumnFamily.NAME.toString(),
+            getMetadata(filePrefix + "/d1/file1.rf", range1),
             new DataFileValue(10, 200).encodeAsString());
         bw1.addMutation(mut);
 
@@ -401,4 +426,20 @@
       }
     }
   }
+
+  private static String getMetadata(String file, Range range) {
+    return StoredTabletFile.of(new Path(file), range).getMetadata();
+  }
+
+  static class RangeArgumentsProvider implements ArgumentsProvider {
+
+    @Override
+    public Stream<? extends Arguments> provideArguments(ExtensionContext context) {
+      return Stream.of(
+          // Pass in up to 3 arguments of infinite ranges to test non-ranged files
+          Arguments.of(new Range(), new Range(), new Range()),
+          // For second run pass in up to 3 arguments with the first two non-infinite ranges
+          Arguments.of(new Range("row_0"), new Range("row_1"), new Range()));
+    }
+  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java b/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java
index 9f0bd56..4dc89ca 100644
--- a/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CloseScannerIT.java
@@ -18,18 +18,19 @@
  */
 package org.apache.accumulo.test;
 
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
 import java.util.List;
 
 import org.apache.accumulo.core.client.Accumulo;
 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.IsolatedScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.accumulo.test.util.Wait;
 import org.junit.jupiter.api.Test;
 
 public class CloseScannerIT extends AccumuloClusterHarness {
@@ -61,16 +62,21 @@
         } // when the scanner is closed, all open sessions should be closed
       }
 
-      List<String> tservers = client.instanceOperations().getTabletServers();
-      int activeScans = 0;
-      for (String tserver : tservers) {
-        activeScans += client.instanceOperations().getActiveScans(tserver).size();
-      }
-
-      assertTrue(activeScans < 3);
+      Wait.waitFor(() -> getActiveScansCount(client) < 1, 5000, 250,
+          "Found active scans after closing all scanners. Expected to find no scans");
     }
   }
 
+  private static int getActiveScansCount(AccumuloClient client)
+      throws AccumuloException, AccumuloSecurityException {
+    List<String> tservers = client.instanceOperations().getTabletServers();
+    int scanCount = 0;
+    for (String tserver : tservers) {
+      scanCount += client.instanceOperations().getActiveScans(tserver).size();
+    }
+    return scanCount;
+  }
+
   private static Scanner createScanner(AccumuloClient client, String tableName, int i)
       throws Exception {
     Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY);
diff --git a/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java b/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java
new file mode 100644
index 0000000..c33c9f9
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java
@@ -0,0 +1,1251 @@
+/*
+ * 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.test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.toMap;
+import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.SortedSet;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+
+import org.apache.accumulo.core.client.Accumulo;
+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.ConditionalWriter;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+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.TableOfflineException;
+import org.apache.accumulo.core.client.admin.CloneConfiguration;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.rfile.RFile;
+import org.apache.accumulo.core.client.sample.Sampler;
+import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.client.security.SecurityErrorCode;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.client.summary.CountingSummarizer;
+import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.Condition;
+import org.apache.accumulo.core.data.ConditionalMutation;
+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.data.constraints.Constraint;
+import org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint;
+import org.apache.accumulo.core.data.constraints.VisibilityConstraint;
+import org.apache.accumulo.core.iterators.Filter;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+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.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.test.util.Wait;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.MoreCollectors;
+
+/**
+ * The purpose of this test is to exercise a large amount of Accumulo's features in a single test.
+ * It provides a quick test that verifies a lot of functionality is working for basic use. This test
+ * does not provide deep coverage of the features it tests.
+ */
+@Tag(SUNNY_DAY)
+public class ComprehensiveIT extends SharedMiniClusterBase {
+
+  public static final String DOG_AND_CAT = "DOG&CAT";
+  static final Authorizations AUTHORIZATIONS = new Authorizations("CAT", "DOG");
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.securityOperations().changeUserAuthorizations("root", AUTHORIZATIONS);
+    }
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testBulkImport() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table);
+
+      bulkImport(client, table, List.of(generateKeys(0, 100), generateKeys(100, 200)));
+
+      verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 200));
+
+      bulkImport(client, table,
+          List.of(generateKeys(200, 300), generateKeys(300, 400), generateKeys(400, 500)));
+
+      verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 500));
+    }
+  }
+
+  @Test
+  public void testMergeAndSplit() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table);
+
+      final SortedMap<Key,Value> expectedData = generateKeys(0, 300);
+
+      write(client, table, generateMutations(0, 300, tr -> true));
+
+      verifyData(client, table, AUTHORIZATIONS, expectedData);
+
+      // test adding splits to a table
+      var splits = new TreeSet<>(List.of(new Text(row(75)), new Text(row(150))));
+      client.tableOperations().addSplits(table, splits);
+      assertEquals(splits, new TreeSet<>(client.tableOperations().listSplits(table)));
+      // adding splits should not change data
+      verifyData(client, table, AUTHORIZATIONS, expectedData);
+
+      // test merging splits away
+      client.tableOperations().merge(table, null, null);
+      assertEquals(Set.of(), new TreeSet<>(client.tableOperations().listSplits(table)));
+      // merging should not change data
+      verifyData(client, table, AUTHORIZATIONS, expectedData);
+
+      splits = new TreeSet<>(List.of(new Text(row(33)), new Text(row(66))));
+      client.tableOperations().addSplits(table, splits);
+      assertEquals(splits, new TreeSet<>(client.tableOperations().listSplits(table)));
+      verifyData(client, table, AUTHORIZATIONS, expectedData);
+
+      client.tableOperations().merge(table, null, null);
+      assertEquals(Set.of(), new TreeSet<>(client.tableOperations().listSplits(table)));
+      verifyData(client, table, AUTHORIZATIONS, expectedData);
+    }
+  }
+
+  @Test
+  public void testFlushAndIterator() throws Exception {
+    String table = getUniqueNames(1)[0];
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table);
+
+      // test attaching an iterator to a table AND flushing a table
+      var iterSetting = new IteratorSetting(200, "fam3", FamFilter.class);
+      iterSetting.addOption("family", "3");
+      client.tableOperations().attachIterator(table, iterSetting,
+          EnumSet.of(IteratorUtil.IteratorScope.minc));
+      final int minRow = 300;
+      final int maxRow = 310;
+      write(client, table, generateMutations(minRow, maxRow, tr -> true));
+      // prior to flushing fam3 should not be filtered by the attached iterator
+      verifyData(client, table, AUTHORIZATIONS, generateKeys(minRow, maxRow));
+      client.tableOperations().flush(table, null, null, true);
+
+      // the attached iterator should be applied when the flush happens filtering out family 3.
+      var expected = generateKeys(minRow, maxRow, tr -> tr.row < minRow || tr.fam != 3);
+
+      // its possible the iterator setting did not make it to the tserver, so wait for that
+      Wait.waitFor(() -> expected.equals(scan(client, table, AUTHORIZATIONS)));
+
+      verifyData(client, table, AUTHORIZATIONS, expected);
+    }
+  }
+
+  @Test
+  public void testConstraint() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table);
+
+      try (var scanner = client.createScanner(table)) {
+        assertEquals(0, scanner.stream().count());
+      }
+
+      // test adding and removing a constraint, do this before taking table online offline so that
+      // test can be sure constraint is picked up
+      assertEquals(Map.of(DefaultKeySizeConstraint.class.getName(), 1),
+          client.tableOperations().listConstraints(table));
+      client.tableOperations().addConstraint(table, TestConstraint.class.getName());
+      assertEquals(
+          Map.of(DefaultKeySizeConstraint.class.getName(), 1, TestConstraint.class.getName(), 2),
+          client.tableOperations().listConstraints(table));
+
+      assertTrue(client.tableOperations().isOnline(table));
+
+      // test taking table offline and then back online
+      client.tableOperations().offline(table, true);
+      assertThrows(TableOfflineException.class, () -> {
+        try (var scanner = client.createScanner(table)) {
+          assertEquals(0, scanner.stream().count());
+        }
+      });
+      assertFalse(client.tableOperations().isOnline(table));
+      client.tableOperations().online(table, true);
+
+      // continue testing constraint, it should cause a write to fail
+      var mre = assertThrows(MutationsRejectedException.class, () -> {
+        try (var writer = client.createBatchWriter(table)) {
+          Mutation m = new Mutation("not a number");
+          m.put("5", "5", "5");
+          writer.addMutation(m);
+        }
+      });
+
+      assertEquals(1, mre.getConstraintViolationSummaries().size());
+      assertEquals("No numeric field seen",
+          mre.getConstraintViolationSummaries().get(0).getViolationDescription());
+
+      // ensure no data was added to table, constraint supposedly prevented previous write
+      try (var scanner = client.createScanner(table)) {
+        assertEquals(0, scanner.stream().count());
+      }
+
+      client.tableOperations().removeConstraint(table, 2);
+      assertEquals(Map.of(DefaultKeySizeConstraint.class.getName(), 1),
+          client.tableOperations().listConstraints(table));
+
+      client.tableOperations().offline(table, true);
+      client.tableOperations().online(table, true);
+
+      try (var writer = client.createBatchWriter(table)) {
+        Mutation m = new Mutation("not a number");
+        m.put("5", "5", "5");
+        writer.addMutation(m);
+      }
+
+      try (var scanner = client.createScanner(table)) {
+        assertEquals(1, scanner.stream().count());
+      }
+
+    }
+  }
+
+  /*
+   * Other tests in this IT show that tservers react to changes in table props, so not testing that
+   * here. Just focusing on the table property APIs.
+   */
+  @Test
+  public void testTableProperties() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table, new NewTableConfiguration().withoutDefaultIterators());
+
+      assertEquals(Map.of(), client.tableOperations().getTableProperties(table));
+
+      client.tableOperations().setProperty(table, "table.custom.compit", "123");
+      Wait.waitFor(() -> Map.of("table.custom.compit", "123")
+          .equals(client.tableOperations().getTableProperties(table)));
+      assertEquals(Map.of("table.custom.compit", "123"),
+          client.tableOperations().getTableProperties(table));
+      assertTrue(
+          client.tableOperations().getConfiguration(table).containsKey("table.custom.compit"));
+
+      client.tableOperations().setProperty(table, "table.custom.compit2", "abc");
+      Wait.waitFor(() -> Map.of("table.custom.compit", "123", "table.custom.compit2", "abc")
+          .equals(client.tableOperations().getTableProperties(table)));
+      assertEquals(Map.of("table.custom.compit", "123", "table.custom.compit2", "abc"),
+          client.tableOperations().getTableProperties(table));
+      assertTrue(client.tableOperations().getConfiguration(table).keySet()
+          .containsAll(Set.of("table.custom.compit", "table.custom.compit2")));
+
+      client.tableOperations().removeProperty(table, "table.custom.compit");
+      Wait.waitFor(() -> Map.of("table.custom.compit2", "abc")
+          .equals(client.tableOperations().getTableProperties(table)));
+      assertEquals(Map.of("table.custom.compit2", "abc"),
+          client.tableOperations().getTableProperties(table));
+      assertTrue(
+          client.tableOperations().getConfiguration(table).containsKey("table.custom.compit2"));
+    }
+  }
+
+  @Test
+  public void testConditionalWriter() throws Exception {
+    String table = getUniqueNames(1)[0];
+    String family = "f1";
+    String qualifier = "q1";
+    String value1 = "v1";
+    String value2 = "v2";
+    String row = row(5);
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table);
+
+      // tests conditional writer
+      try (var condWriter = client.createConditionalWriter(table)) {
+        try (var scanner = client.createScanner(table)) {
+          scanner.setRange(new Range(row));
+          scanner.fetchColumn(family, qualifier);
+          assertEquals(0, scanner.stream().count());
+        }
+
+        var cMutation = new ConditionalMutation(row);
+        cMutation.addCondition(new Condition(family, qualifier).setValue(value1));
+        cMutation.put(family, qualifier, value2);
+        assertEquals(ConditionalWriter.Status.REJECTED, condWriter.write(cMutation).getStatus());
+
+        try (var scanner = client.createScanner(table)) {
+          scanner.setRange(new Range(row));
+          scanner.fetchColumn(family, qualifier);
+          assertEquals(0, scanner.stream().count());
+        }
+
+        cMutation = new ConditionalMutation(row);
+        cMutation.addCondition(new Condition(family, qualifier));
+        cMutation.put(family, qualifier, value1);
+        assertEquals(ConditionalWriter.Status.ACCEPTED, condWriter.write(cMutation).getStatus());
+
+        // ensure table was changed
+        try (var scanner = client.createScanner(table)) {
+          scanner.setRange(new Range(row));
+          // tests scanner method to fetch a column family and qualifier
+          scanner.fetchColumn(family, qualifier);
+          assertEquals(1, scanner.stream().count());
+        }
+
+        cMutation = new ConditionalMutation(row);
+        cMutation.addCondition(new Condition(family, qualifier).setValue(value1));
+        cMutation.putDelete(family, qualifier);
+        assertEquals(ConditionalWriter.Status.ACCEPTED, condWriter.write(cMutation).getStatus());
+
+        // ensure table was changed
+        try (var scanner = client.createScanner(table)) {
+          scanner.setRange(new Range(row));
+          scanner.fetchColumn(family, qualifier);
+          assertEquals(0, scanner.stream().count());
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testOverwriteAndBatchDeleter() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table);
+
+      write(client, table, generateMutations(0, 100, tr -> true));
+
+      verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 100, tr -> true));
+
+      write(client, table, generateMutations(0, 50, 0x7abc1234, tr -> true));
+
+      // test the test
+      assertNotEquals(generateKeys(0, 50, 0x7abc1234, tr -> true), generateKeys(0, 50));
+
+      TreeMap<Key,Value> expected = new TreeMap<>();
+      expected.putAll(generateKeys(0, 50, 0x7abc1234, tr -> true));
+      expected.putAll(generateKeys(50, 100));
+
+      verifyData(client, table, AUTHORIZATIONS, expected);
+
+      bulkImport(client, table, List.of(generateKeys(25, 75, 0x12345678, tr -> true),
+          generateKeys(90, 200, 0x76543210, tr -> true)));
+
+      expected.putAll(generateKeys(25, 75, 0x12345678, tr -> true));
+      expected.putAll(generateKeys(90, 200, 0x76543210, tr -> true));
+
+      verifyData(client, table, AUTHORIZATIONS, expected);
+
+      Range delRange1 = new Range(row(20), row(59));
+      Range delRange2 = new Range(row(65), row(91));
+
+      try (var deleter = client.createBatchDeleter(table, AUTHORIZATIONS, 3)) {
+        deleter.setRanges(List.of(delRange1, delRange2));
+        deleter.delete();
+      }
+
+      int sizeBefore = expected.size();
+
+      expected.keySet().removeIf(k -> delRange1.contains(k) || delRange2.contains(k));
+
+      assertTrue(sizeBefore > expected.size());
+
+      verifyData(client, table, AUTHORIZATIONS, expected);
+
+      client.tableOperations().compact(table, new CompactionConfig().setWait(true));
+
+      verifyData(client, table, AUTHORIZATIONS, expected);
+    }
+  }
+
+  @Test
+  public void invalidInstanceName() {
+    try (var client = Accumulo.newClient().to("fake_instance_name", getCluster().getZooKeepers())
+        .as(getAdminPrincipal(), getToken()).build()) {
+      assertThrows(RuntimeException.class, () -> client.instanceOperations().getTabletServers());
+    }
+  }
+
+  @Test
+  public void testMultiTableWrite() throws Exception {
+    String[] tables = getUniqueNames(2);
+    String table1 = tables[0];
+    String table2 = tables[1];
+
+    try (var client = Accumulo.newClient().from(getClientProps()).build()) {
+      client.tableOperations().create(table1);
+      client.tableOperations().create(table2);
+
+      try (var writer = client.createMultiTableBatchWriter()) {
+        writer.getBatchWriter(table1).addMutations(generateMutations(0, 100, tr -> true));
+        writer.getBatchWriter(table2).addMutations(generateMutations(100, 200, tr -> true));
+        writer.getBatchWriter(table1).addMutations(generateMutations(200, 300, tr -> true));
+        writer.getBatchWriter(table2).addMutations(generateMutations(300, 400, tr -> true));
+      }
+
+      TreeMap<Key,Value> expected1 = new TreeMap<>();
+      expected1.putAll(generateKeys(0, 100));
+      expected1.putAll(generateKeys(200, 300));
+
+      TreeMap<Key,Value> expected2 = new TreeMap<>();
+      expected2.putAll(generateKeys(100, 200));
+      expected2.putAll(generateKeys(300, 400));
+
+      verifyData(client, table1, AUTHORIZATIONS, expected1);
+      verifyData(client, table2, AUTHORIZATIONS, expected2);
+
+      try (var writer = client.createMultiTableBatchWriter()) {
+        writer.getBatchWriter(table1)
+            .addMutations(generateMutations(0, 100, 0x12345678, tr -> true));
+        writer.getBatchWriter(table2)
+            .addMutations(generateMutations(100, 200, 0x12345678, tr -> true));
+        writer.getBatchWriter(table1)
+            .addMutations(generateMutations(200, 300, 0x12345678, tr -> true));
+        writer.getBatchWriter(table2)
+            .addMutations(generateMutations(300, 400, 0x12345678, tr -> true));
+      }
+
+      expected1.putAll(generateKeys(0, 100, 0x12345678, tr -> true));
+      expected1.putAll(generateKeys(200, 300, 0x12345678, tr -> true));
+      expected2.putAll(generateKeys(100, 200, 0x12345678, tr -> true));
+      expected2.putAll(generateKeys(300, 400, 0x12345678, tr -> true));
+
+      verifyData(client, table1, AUTHORIZATIONS, expected1);
+      verifyData(client, table2, AUTHORIZATIONS, expected2);
+    }
+  }
+
+  @Test
+  public void testSecurity() throws Exception {
+    String[] tables = getUniqueNames(2);
+    String rootsTable = tables[0];
+    String usersTable = tables[0];
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      client.tableOperations().create(rootsTable);
+      write(client, rootsTable, generateMutations(0, 100, tr -> true));
+      verifyData(client, rootsTable, AUTHORIZATIONS, generateKeys(0, 100));
+
+      var password = new PasswordToken("bestpass1234");
+      client.securityOperations().createLocalUser("user1", password);
+
+      try (var userClient =
+          Accumulo.newClient().from(getClientProps()).as("user1", password).build()) {
+
+        // user1 should not be able to read table
+        var ise = assertThrows(IllegalStateException.class,
+            () -> verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 100)));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED,
+            ((AccumuloSecurityException) ise.getCause()).getSecurityErrorCode());
+
+        // user1 should not be able to grant theirself read access
+        var ase = assertThrows(AccumuloSecurityException.class, () -> userClient
+            .securityOperations().grantTablePermission("user1", rootsTable, TablePermission.READ));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+
+        // grant user1 read access
+        client.securityOperations().grantTablePermission("user1", rootsTable, TablePermission.READ);
+
+        // security changes take time to propagate, should eventually be able to scan table
+        Wait.waitFor(() -> {
+          try {
+            verifyData(userClient, rootsTable, Authorizations.EMPTY,
+                generateKeys(0, 100, tr -> tr.vis.isEmpty()));
+            return true;
+          } catch (IllegalStateException e) {
+            return false;
+          }
+        });
+        verifyData(userClient, rootsTable, Authorizations.EMPTY,
+            generateKeys(0, 100, tr -> tr.vis.isEmpty()));
+
+        // should not be able to scan with authorizations the user does not have
+        ise = assertThrows(IllegalStateException.class,
+            () -> verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 100)));
+        assertEquals(SecurityErrorCode.BAD_AUTHORIZATIONS,
+            ((AccumuloSecurityException) ise.getCause()).getSecurityErrorCode());
+
+        // scan w/o setting auths, should only return data w/o auths
+        try (Scanner scanner = userClient.createScanner(rootsTable)) {
+          assertEquals(generateKeys(0, 100, tr -> tr.vis.isEmpty()), scan(scanner));
+        }
+
+        // user should not have permission to write to table
+        var mre = assertThrows(MutationsRejectedException.class,
+            () -> write(userClient, rootsTable, generateMutations(100, 200, tr -> true)));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, mre.getSecurityErrorCodes().values()
+            .stream().flatMap(Set::stream).collect(MoreCollectors.onlyElement()));
+        // ensure no new data was written
+        assertEquals(new Text(row(99)), client.tableOperations().getMaxRow(rootsTable,
+            AUTHORIZATIONS, new Text(row(98)), true, new Text(row(110)), true));
+
+        client.securityOperations().grantTablePermission("user1", rootsTable,
+            TablePermission.WRITE);
+        // security changes take time to propagate, should eventually be able to write
+        Wait.waitFor(() -> {
+          try {
+            write(userClient, rootsTable, generateMutations(100, 200, tr -> true));
+            return true;
+          } catch (MutationsRejectedException e) {
+            return false;
+          }
+        });
+
+        // ensure newly written data is visible
+        verifyData(client, rootsTable, AUTHORIZATIONS, generateKeys(0, 200));
+
+        // allow user to write to table and verify can write
+        client.securityOperations().changeUserAuthorizations("user1", AUTHORIZATIONS);
+        Wait.waitFor(() -> {
+          try {
+            verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 200));
+            return true;
+          } catch (IllegalStateException e) {
+            return false;
+          }
+        });
+        verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 200));
+        // should scan with all of users granted auths since no auths were specified
+        try (Scanner scanner = userClient.createScanner(rootsTable)) {
+          assertEquals(generateKeys(0, 200), scan(scanner));
+        }
+
+        var splits = new TreeSet<>(List.of(new Text(row(50))));
+
+        // should not have permission to alter the table
+        ase = assertThrows(AccumuloSecurityException.class,
+            () -> userClient.tableOperations().addSplits(rootsTable, splits));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+        // ensure no splits were added
+        assertEquals(Set.of(), Set.copyOf(client.tableOperations().listSplits(rootsTable)));
+
+        client.securityOperations().grantTablePermission("user1", rootsTable,
+            TablePermission.ALTER_TABLE);
+        Wait.waitFor(() -> {
+          try {
+            userClient.tableOperations().addSplits(rootsTable, splits);
+            return true;
+          } catch (AccumuloSecurityException e) {
+            return false;
+          }
+        });
+        assertEquals(splits, Set.copyOf(userClient.tableOperations().listSplits(rootsTable)));
+
+        // user should not have permission to bulk import
+        ase = assertThrows(AccumuloSecurityException.class, () -> bulkImport(userClient, rootsTable,
+            List.of(generateKeys(200, 250, tr -> true), generateKeys(250, 300, tr -> true))));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+        verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 200));
+
+        // TODO open a bug about this, had to add this permission to get bulk import to work
+        client.securityOperations().grantSystemPermission("user1", SystemPermission.SYSTEM);
+        // give permission to bulk import and verify it works
+        client.securityOperations().grantTablePermission("user1", rootsTable,
+            TablePermission.BULK_IMPORT);
+        Wait.waitFor(() -> {
+          try {
+            bulkImport(userClient, rootsTable,
+                List.of(generateKeys(200, 250, tr -> true), generateKeys(250, 300, tr -> true)));
+            return true;
+          } catch (AccumuloSecurityException e) {
+            return false;
+          }
+        });
+        verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 300));
+        client.securityOperations().revokeSystemPermission("user1", SystemPermission.SYSTEM);
+
+        // user1 should not be able to delete the table
+        ase = assertThrows(AccumuloSecurityException.class,
+            () -> userClient.tableOperations().delete(rootsTable));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+        // table should still exists and be readable
+        verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 300));
+
+        // remove user1 table permission and veriy that eventually they can not read
+        client.securityOperations().revokeTablePermission("user1", rootsTable,
+            TablePermission.READ);
+        Wait.waitFor(() -> {
+          try {
+            verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 300));
+            return false;
+          } catch (IllegalStateException e) {
+            assertEquals(SecurityErrorCode.PERMISSION_DENIED,
+                ((AccumuloSecurityException) e.getCause()).getSecurityErrorCode());
+            return true;
+          }
+        });
+
+        // grant user1 permissions to drop table, delete the table and verify its deleted
+        client.securityOperations().grantTablePermission("user1", rootsTable,
+            TablePermission.DROP_TABLE);
+        Wait.waitFor(() -> {
+          try {
+            userClient.tableOperations().delete(rootsTable);
+            return true;
+          } catch (AccumuloSecurityException e) {
+            return false;
+          }
+        });
+        assertThrows(TableNotFoundException.class,
+            () -> verifyData(userClient, rootsTable, AUTHORIZATIONS, generateKeys(0, 300)));
+        assertFalse(userClient.tableOperations().list().contains(rootsTable));
+
+        // user1 should not be able to create a table
+        ase = assertThrows(AccumuloSecurityException.class,
+            () -> userClient.tableOperations().create(usersTable));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+
+        // create namespace and grant user1 access to create tables in the namespace
+        client.namespaceOperations().create("ns1");
+        client.securityOperations().grantNamespacePermission("user1", "ns1",
+            NamespacePermission.CREATE_TABLE);
+        var tableNS = "ns1." + usersTable;
+        Wait.waitFor(() -> {
+          try {
+            var ntc = new NewTableConfiguration()
+                .setProperties(Map.of(Property.TABLE_CONSTRAINT_PREFIX.getKey() + "2",
+                    VisibilityConstraint.class.getName()));
+            userClient.tableOperations().create(tableNS, ntc);
+            return true;
+          } catch (AccumuloSecurityException e) {
+            return false;
+          }
+        });
+
+        // user1 should still not be able to create table in the default namepsace
+        ase = assertThrows(AccumuloSecurityException.class,
+            () -> userClient.tableOperations().create(usersTable));
+        assertEquals(SecurityErrorCode.PERMISSION_DENIED, ase.getSecurityErrorCode());
+
+        // verify user1 can interact with table they created
+        write(userClient, tableNS, generateMutations(0, 100, tr -> true));
+        verifyData(userClient, tableNS, AUTHORIZATIONS, generateKeys(0, 100, tr -> true));
+
+        // confirm user can not write data they can not see because visibility constraint was set on
+        // table
+        mre = assertThrows(MutationsRejectedException.class, () -> {
+          try (var writer = userClient.createBatchWriter(tableNS)) {
+            Mutation m = new Mutation("invisible");
+            m.put("f", "q", new ColumnVisibility("DOG&HAMSTER"), "v1");
+            writer.addMutation(m);
+          }
+        });
+        assertEquals(VisibilityConstraint.class.getName(), mre.getConstraintViolationSummaries()
+            .stream().map(cvs -> cvs.constrainClass).collect(MoreCollectors.onlyElement()));
+
+        // confirm user can delete table
+        userClient.tableOperations().delete(tableNS);
+        assertFalse(userClient.tableOperations().list().contains(tableNS));
+        assertFalse(userClient.tableOperations().list().contains(usersTable));
+      }
+
+      // attempt to perform operations with incorrect password
+      try (var userClient = Accumulo.newClient().from(getClientProps())
+          .as("user1", new PasswordToken("bestpass123")).build()) {
+        var ase = assertThrows(AccumuloSecurityException.class,
+            () -> userClient.tableOperations().create("ns1." + usersTable));
+        assertEquals(SecurityErrorCode.BAD_CREDENTIALS, ase.getSecurityErrorCode());
+      }
+
+    }
+  }
+
+  /*
+   * This test happens to cover a lot features in the Accumulo public API like sampling,
+   * summarizations, and some table operations. Those features do not need to be tested elsewhere.
+   */
+  @Test
+  public void testCreateTableWithManyOptions() throws Exception {
+    String[] tables = getUniqueNames(3);
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      var everythingTable = tables[0];
+      var everythingClone = tables[1];
+      var everythingImport = tables[1];
+
+      // create a table with a lot of initial config
+      client.tableOperations().create(everythingTable, getEverythingTableConfig());
+
+      write(client, everythingTable, generateMutations(0, 100, tr -> true));
+
+      verifyEverythingTable(client, everythingTable);
+
+      // test cloning a table as part of this test because the table has lots of customizations.
+      client.tableOperations().clone(everythingTable, everythingClone,
+          CloneConfiguration.builder().setFlush(true).build());
+
+      // check the clone has all the same config and data as the original table
+      verifyEverythingTable(client, everythingClone);
+
+      // test compaction with an iterator that filters out col fam 3
+      CompactionConfig compactionConfig = new CompactionConfig();
+      var iterSetting = new IteratorSetting(200, "fam3", FamFilter.class);
+      iterSetting.addOption("family", "3");
+      compactionConfig.setIterators(List.of(iterSetting)).setWait(true);
+      client.tableOperations().compact(everythingClone, compactionConfig);
+      // scans should not see col fam 3 now
+      verifyData(client, everythingClone, AUTHORIZATIONS,
+          generateKeys(0, 100, tr -> tr.fam != 3 && tr.fam != 9));
+      verifyData(client, everythingClone, AUTHORIZATIONS,
+          generateKeys(3, 4, tr -> tr.fam != 3 && tr.fam != 9),
+          scanner -> scanner.setSamplerConfiguration(everythingSampleConfig));
+
+      // remove the iterator that is suppressing col fam 9
+      client.tableOperations().removeIterator(everythingClone, "fam9",
+          EnumSet.of(IteratorUtil.IteratorScope.scan));
+      Wait.waitFor(
+          () -> !client.tableOperations().listIterators(everythingClone).containsKey("fam9"));
+      assertFalse(client.tableOperations().listIterators(everythingClone).containsKey("fam9"));
+
+      var expected = generateKeys(0, 100, tr -> tr.fam != 3);
+      // the iterator removal may not have made it to the tserver, so wait for it
+      Wait.waitFor(() -> expected.equals(scan(client, everythingClone, AUTHORIZATIONS)));
+
+      verifyData(client, everythingClone, AUTHORIZATIONS, expected);
+
+      // test deleting a row ranges and scanning to verify its gone
+      client.tableOperations().deleteRows(everythingClone, new Text(row(35)), new Text(row(40)));
+      verifyData(client, everythingClone, AUTHORIZATIONS,
+          generateKeys(0, 100, tr -> (tr.row <= 35 || tr.row > 40) && tr.fam != 3));
+
+      // the changes to the clone should not have affected the source table so verify it again
+      verifyEverythingTable(client, everythingTable);
+
+      // test renaming a table
+      String tableIdBeforeRename = client.tableOperations().tableIdMap().get(everythingClone);
+      client.tableOperations().rename(everythingClone, everythingClone + "9");
+      assertFalse(client.tableOperations().list().contains(everythingClone));
+
+      // renaming a table should not affect its data
+      verifyData(client, everythingClone + "9", AUTHORIZATIONS,
+          generateKeys(0, 100, tr -> (tr.row <= 35 || tr.row > 40) && tr.fam != 3));
+      String tableIdAfterRename = client.tableOperations().tableIdMap().get(everythingClone + "9");
+      assertEquals(tableIdBeforeRename, tableIdAfterRename);
+
+      // test deleting a table
+      client.tableOperations().delete(everythingClone + "9");
+      assertFalse(client.tableOperations().list().contains(everythingClone + "9"));
+
+      // test exporting and importing a table as part of this test because the table has lots of
+      // customizations.
+      exportImport(client, everythingTable, everythingImport);
+
+      verifyEverythingTable(client, everythingImport);
+    }
+  }
+
+  @Test
+  public void testNamespaces() throws Exception {
+    var namespace = "compns";
+    var uniq = getUniqueNames(1)[0];
+    String table = namespace + "." + uniq;
+    String table2 = namespace + "2." + uniq;
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      assertFalse(client.namespaceOperations().exists(namespace));
+
+      // creation of table should fail when namespace does not exists
+      assertThrows(AccumuloException.class, () -> client.tableOperations().create(table));
+
+      client.namespaceOperations().create(namespace);
+
+      client.tableOperations().create(table, new NewTableConfiguration().withoutDefaultIterators());
+
+      assertTrue(client.namespaceOperations().list().contains(namespace));
+      assertTrue(client.namespaceOperations().exists(namespace));
+
+      client.namespaceOperations().setProperty(namespace, "table.custom.p1", "v1");
+      client.tableOperations().setProperty(table, "table.custom.p2", "v2");
+
+      Wait.waitFor(() -> client.namespaceOperations().getNamespaceProperties(namespace)
+          .containsKey("table.custom.p1"));
+
+      assertEquals(Map.of("table.custom.p1", "v1"),
+          client.namespaceOperations().getNamespaceProperties(namespace));
+      assertEquals(Map.of("table.custom.p2", "v2"),
+          client.tableOperations().getTableProperties(table));
+      assertTrue(client.tableOperations().getConfiguration(table).keySet()
+          .containsAll(Set.of("table.custom.p1", "table.custom.p2")));
+
+      client.namespaceOperations().removeProperty(namespace, "table.custom.p1");
+      // should not impact table prop
+      client.namespaceOperations().removeProperty(namespace, "table.custom.p2");
+
+      Wait.waitFor(() -> !client.namespaceOperations().getNamespaceProperties(namespace)
+          .containsKey("table.custom.p1"));
+
+      assertEquals(Map.of(), client.namespaceOperations().getNamespaceProperties(namespace));
+      assertEquals(Map.of("table.custom.p2", "v2"),
+          client.tableOperations().getTableProperties(table));
+      assertTrue(client.tableOperations().getConfiguration(table).containsKey("table.custom.p2"));
+
+      client.namespaceOperations().rename(namespace, namespace + "2");
+
+      assertFalse(client.namespaceOperations().exists(namespace));
+      assertTrue(client.namespaceOperations().exists(namespace + "2"));
+      assertFalse(client.tableOperations().exists(table));
+      assertTrue(client.tableOperations().exists(table2));
+
+      assertThrows(NamespaceNotEmptyException.class,
+          () -> client.namespaceOperations().delete(namespace + "2"));
+
+      client.tableOperations().delete(table2);
+      client.namespaceOperations().delete(namespace + "2");
+      assertFalse(client.namespaceOperations().exists(namespace + "2"));
+    }
+  }
+
+  private static final SortedSet<Text> everythingSplits =
+      new TreeSet<>(List.of(new Text(row(33)), new Text(row(66))));
+  private static final Map<String,Set<Text>> everythingLocalityGroups =
+      Map.of("G1", Set.of(new Text(family(3)), new Text(family(7))));
+  private static final SamplerConfiguration everythingSampleConfig =
+      new SamplerConfiguration(RowThreeSampler.class.getName());
+
+  private static NewTableConfiguration getEverythingTableConfig() {
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.withSplits(everythingSplits);
+    ntc.setProperties(Map.of(Property.TABLE_DURABILITY.getKey(), "flush"));
+    ntc.setTimeType(TimeType.LOGICAL);
+    ntc.enableSummarization(SummarizerConfiguration.builder(FamilySummarizer.class).build());
+    ntc.enableSampling(everythingSampleConfig);
+    ntc.setLocalityGroups(everythingLocalityGroups);
+    var iterSetting = new IteratorSetting(100, "fam9", FamFilter.class);
+    iterSetting.addOption("family", "9");
+    ntc.attachIterator(iterSetting, EnumSet.of(IteratorUtil.IteratorScope.scan));
+    return ntc;
+  }
+
+  private static void verifyEverythingTable(AccumuloClient client, String table) throws Exception {
+    assertEquals(TimeType.LOGICAL, client.tableOperations().getTimeType(table));
+    assertEquals(everythingSampleConfig, client.tableOperations().getSamplerConfiguration(table));
+    assertTrue(client.tableOperations().tableIdMap().keySet().contains(table));
+    assertTrue(client.tableOperations().list().contains(table));
+    assertEquals(everythingSplits, new TreeSet<>(client.tableOperations().listSplits(table)));
+    assertTrue(client.tableOperations().exists(table));
+    assertFalse(client.tableOperations().exists("61dc5ad6a4983abaf107e94321f3a37e37375267"));
+    assertEquals(everythingLocalityGroups, client.tableOperations().getLocalityGroups(table));
+    assertEquals(EnumSet.of(IteratorUtil.IteratorScope.scan),
+        client.tableOperations().listIterators(table).get("fam9"));
+    var iterSetting = new IteratorSetting(100, "fam9", FamFilter.class);
+    iterSetting.addOption("family", "9");
+    assertEquals(iterSetting, client.tableOperations().getIteratorSetting(table, "fam9",
+        IteratorUtil.IteratorScope.scan));
+
+    verifyData(client, table, Authorizations.EMPTY,
+        generateKeys(0, 100, tr -> tr.fam != 9 && tr.vis.isEmpty()));
+    verifyData(client, table, new Authorizations("CAT"),
+        generateKeys(0, 100, tr -> tr.fam != 9 && !tr.vis.equals(DOG_AND_CAT)));
+    verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 100, tr -> tr.fam != 9));
+
+    // test setting range on scanner
+    verifyData(client, table, Authorizations.EMPTY,
+        generateKeys(6, 19, tr -> tr.fam == 8 && tr.vis.isEmpty()), scanner -> {
+          if (scanner instanceof Scanner) {
+            ((Scanner) scanner).setRange(new Range(row(6), row(18)));
+          } else if (scanner instanceof BatchScanner) {
+            ((BatchScanner) scanner).setRanges(List.of(new Range(row(6), row(18))));
+          } else {
+            throw new IllegalArgumentException();
+          }
+
+          scanner.fetchColumnFamily(new Text(family(8)));
+        });
+
+    try (var scanner = client.createBatchScanner(table, AUTHORIZATIONS)) {
+      // set multiple ranges on scanner
+      scanner
+          .setRanges(List.of(new Range(row(6), row(18)), new Range(row(27), true, row(37), false)));
+      assertEquals(generateKeys(6, 37, tr -> tr.fam != 9 && (tr.row <= 18 || tr.row >= 27)),
+          scan(scanner));
+    }
+
+    // test scanning sample data
+    verifyData(client, table, Authorizations.EMPTY,
+        generateKeys(3, 4, tr -> tr.fam != 9 && tr.vis.isEmpty()),
+        scanner -> scanner.setSamplerConfiguration(everythingSampleConfig));
+    verifyData(client, table, new Authorizations("CAT"),
+        generateKeys(3, 4, tr -> tr.fam != 9 && !tr.vis.equals(DOG_AND_CAT)),
+        scanner -> scanner.setSamplerConfiguration(everythingSampleConfig));
+    verifyData(client, table, AUTHORIZATIONS, generateKeys(3, 4, tr -> tr.fam != 9),
+        scanner -> scanner.setSamplerConfiguration(everythingSampleConfig));
+
+    // test fetching column families
+    verifyData(client, table, Authorizations.EMPTY,
+        generateKeys(0, 100, tr -> tr.fam == 5 && tr.vis.isEmpty()),
+        scanner -> scanner.fetchColumnFamily(new Text(family(5))));
+    verifyData(client, table, new Authorizations("CAT"),
+        generateKeys(0, 100, tr -> tr.fam == 5 && !tr.vis.equals(DOG_AND_CAT)),
+        scanner -> scanner.fetchColumnFamily(new Text(family(5))));
+    verifyData(client, table, AUTHORIZATIONS, generateKeys(0, 100, tr -> tr.fam == 5),
+        scanner -> scanner.fetchColumnFamily(new Text(family(5))));
+
+    // test setting an iterator on a scanner
+    var scanIter = new IteratorSetting(200, "fam7", FamFilter.class);
+    scanIter.addOption("family", "7");
+    verifyData(client, table, Authorizations.EMPTY,
+        generateKeys(0, 100, tr -> tr.fam != 9 && tr.fam != 7 && tr.vis.isEmpty()),
+        scanner -> scanner.addScanIterator(scanIter));
+    verifyData(client, table, new Authorizations("CAT"),
+        generateKeys(0, 100, tr -> tr.fam != 9 && tr.fam != 7 && !tr.vis.equals(DOG_AND_CAT)),
+        scanner -> scanner.addScanIterator(scanIter));
+    verifyData(client, table, AUTHORIZATIONS,
+        generateKeys(0, 100, tr -> tr.fam != 9 && tr.fam != 7),
+        scanner -> scanner.addScanIterator(scanIter));
+
+    // summary data does not exist until flushed
+    client.tableOperations().flush(table, null, null, true);
+    var summaries = client.tableOperations().summaries(table).retrieve();
+    assertEquals(1, summaries.size());
+    for (int i = 0; i < 10; i++) {
+      assertEquals(300, summaries.get(0).getStatistics().get("c:" + family(i)));
+    }
+    assertEquals(10,
+        summaries.get(0).getStatistics().keySet().stream().filter(k -> k.startsWith("c:")).count());
+  }
+
+  private static void write(AccumuloClient client, String table, Collection<Mutation> mutations)
+      throws Exception {
+
+    try (var writer = client.createBatchWriter(table)) {
+      for (var mutation : mutations) {
+        writer.addMutation(mutation);
+      }
+    }
+
+  }
+
+  private static void verifyData(AccumuloClient client, String table, Authorizations auths,
+      SortedMap<Key,Value> expectedData) throws Exception {
+    try (var scanner = client.createScanner(table, auths)) {
+      assertEquals(expectedData, scan(scanner));
+    }
+
+    try (var scanner = client.createBatchScanner(table, auths)) {
+      scanner.setRanges(List.of(new Range()));
+      assertEquals(expectedData, scan(scanner));
+    }
+  }
+
+  private static void verifyData(AccumuloClient client, String table, Authorizations auths,
+      SortedMap<Key,Value> expectedData, Consumer<ScannerBase> scannerConsumer) throws Exception {
+    try (var scanner = client.createScanner(table, auths)) {
+      scannerConsumer.accept(scanner);
+      assertEquals(expectedData, scan(scanner));
+    }
+
+    try (var scanner = client.createBatchScanner(table, auths)) {
+      scanner.setRanges(List.of(new Range()));
+      scannerConsumer.accept(scanner);
+      assertEquals(expectedData, scan(scanner));
+    }
+  }
+
+  private SortedMap<Key,Value> scan(AccumuloClient client, String table, Authorizations auths)
+      throws TableNotFoundException {
+    try (var scanner = client.createScanner(table, auths)) {
+      return scan(scanner);
+    }
+  }
+
+  private static SortedMap<Key,Value> scan(ScannerBase scanner) {
+    return scanner.stream().map(e -> {
+      Key nk = new Key(e.getKey());
+      nk.setTimestamp(Long.MAX_VALUE);
+      return new AbstractMap.SimpleEntry<>(nk, e.getValue());
+    }).collect(toMap(Map.Entry::getKey, Map.Entry::getValue, (v1, v2) -> v1, TreeMap::new));
+  }
+
+  private static void bulkImport(AccumuloClient client, String table,
+      List<SortedMap<Key,Value>> data) throws Exception {
+    String tmp = getCluster().getTemporaryPath().toString();
+    Path dir = new Path(tmp, "comp_bulk_" + UUID.randomUUID());
+
+    getCluster().getFileSystem().mkdirs(dir);
+
+    try {
+      int count = 0;
+      for (var keyValues : data) {
+        try (var output = getCluster().getFileSystem().create(new Path(dir, "f" + count + ".rf"));
+            var writer = RFile.newWriter().to(output).build()) {
+          writer.startDefaultLocalityGroup();
+          for (Map.Entry<Key,Value> entry : keyValues.entrySet()) {
+            writer.append(entry.getKey(), entry.getValue());
+          }
+        }
+        count++;
+      }
+
+      client.tableOperations().importDirectory(dir.toString()).to(table).load();
+    } finally {
+      getCluster().getFileSystem().delete(dir, true);
+    }
+  }
+
+  static class TestRecord {
+    final int row;
+    final int fam;
+    final int qual;
+    final String vis;
+    final int val;
+
+    TestRecord(int row, int fam, int qual, String vis, int val) {
+      this.row = row;
+      this.fam = fam;
+      this.qual = qual;
+      this.vis = vis;
+      this.val = val;
+    }
+  }
+
+  static String row(int r) {
+    return String.format("%06d", r);
+  }
+
+  static String family(int f) {
+    return String.format("%04d", f);
+  }
+
+  static String qualifier(int q) {
+    return String.format("%04d", q);
+  }
+
+  static String value(int v) {
+    return String.format("%09d", v);
+  }
+
+  static SortedMap<Key,Value> generateKeys(int minRow, int maxRow) {
+    return generateKeys(minRow, maxRow, tr -> true);
+  }
+
+  static SortedMap<Key,Value> generateKeys(int minRow, int maxRow,
+      Predicate<TestRecord> predicate) {
+    return generateKeys(minRow, maxRow, 0, predicate);
+  }
+
+  static SortedMap<Key,Value> generateKeys(int minRow, int maxRow, int salt,
+      Predicate<TestRecord> predicate) {
+    TreeMap<Key,Value> data = new TreeMap<>();
+    var mutations = generateMutations(minRow, maxRow, salt, predicate);
+    for (Mutation m : mutations) {
+      for (var cu : m.getUpdates()) {
+        Key k = new Key(m.getRow(), cu.getColumnFamily(), cu.getColumnQualifier(),
+            cu.getColumnVisibility());
+        Value v = new Value(cu.getValue());
+        data.put(k, v);
+      }
+    }
+
+    return data;
+  }
+
+  static Collection<Mutation> generateMutations(int minRow, int maxRow,
+      Predicate<TestRecord> predicate) {
+    return generateMutations(minRow, maxRow, 0, predicate);
+  }
+
+  static Collection<Mutation> generateMutations(int minRow, int maxRow, int salt,
+      Predicate<TestRecord> predicate) {
+
+    List<Mutation> mutations = new ArrayList<>();
+
+    for (int r = minRow; r < maxRow; r++) {
+      Mutation m = new Mutation(row(r));
+      for (int f = 0; f < 10; f++) {
+        for (int q = 17; q < 20; q++) {
+          String vis = "";
+          int unsaltedVal = r << 16 | f << 8 | q;
+          if (unsaltedVal % 5 == 0) {
+            vis = DOG_AND_CAT;
+          } else if (unsaltedVal % 11 == 0) {
+            vis = "DOG|CAT";
+          }
+
+          int v = unsaltedVal ^ salt;
+
+          if (predicate.test(new TestRecord(r, f, q, vis, v))) {
+            m.put(family(f), qualifier(q), new ColumnVisibility(vis), value(v));
+          }
+        }
+      }
+      if (m.size() > 0) {
+        mutations.add(m);
+      }
+    }
+
+    return mutations;
+  }
+
+  private void exportImport(AccumuloClient client, String srcTable, String importTable)
+      throws Exception {
+    client.tableOperations().offline(srcTable, true);
+
+    String tmp = getCluster().getTemporaryPath().toString();
+    String exportDir = tmp + "/export_" + srcTable;
+    String importDir = tmp + "/import_" + importTable;
+
+    var fs = getCluster().getFileSystem();
+
+    client.tableOperations().exportTable(srcTable, exportDir);
+
+    fs.mkdirs(new Path(importDir));
+    try (var inputStream = fs.open(new Path(exportDir + "/distcp.txt"));
+        var inputStreamReader = new InputStreamReader(inputStream);
+        var reader = new BufferedReader(inputStreamReader)) {
+      String line;
+      while ((line = reader.readLine()) != null) {
+        var srcPath = new Path(line);
+        Path destPath = new Path(importDir, srcPath.getName());
+        FileUtil.copy(fs, srcPath, fs, destPath, false, fs.getConf());
+      }
+    }
+
+    client.tableOperations().importTable(importTable, importDir);
+
+    client.tableOperations().online(srcTable, true);
+  }
+
+  public static class FamFilter extends Filter {
+
+    private String family = null;
+
+    @Override
+    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options,
+        IteratorEnvironment env) throws IOException {
+      super.init(source, options, env);
+      family = family(Integer.parseInt(options.get("family")));
+    }
+
+    @Override
+    public boolean accept(Key k, Value v) {
+      return !k.getColumnFamilyData().toString().equals(family);
+    }
+  }
+
+  public static class RowThreeSampler implements Sampler {
+
+    @Override
+    public void init(SamplerConfiguration config) {
+
+    }
+
+    @Override
+    public boolean accept(Key k) {
+      return k.getRowData().toString().equals(row(3));
+    }
+  }
+
+  public static class FamilySummarizer extends CountingSummarizer<String> {
+
+    @Override
+    protected Converter<String> converter() {
+      return (k, v, consumer) -> consumer.accept(k.getColumnFamilyData().toString());
+    }
+  }
+
+  public static class TestConstraint implements Constraint {
+
+    @Override
+    public String getViolationDescription(short violationCode) {
+      if (violationCode == 1) {
+        return "No numeric field seen";
+      }
+
+      return null;
+    }
+
+    private boolean isNumber(byte[] field) {
+      try {
+        Integer.parseInt(new String(field, UTF_8));
+        return true;
+      } catch (NumberFormatException e) {
+        return false;
+      }
+    }
+
+    @Override
+    public List<Short> check(Environment env, Mutation mutation) {
+      if (!isNumber(mutation.getRow())) {
+        return List.of((short) 1);
+      }
+
+      for (ColumnUpdate cu : mutation.getUpdates()) {
+        if (!isNumber(cu.getColumnFamily()) || !isNumber(cu.getColumnQualifier())) {
+          return List.of((short) 1);
+        }
+      }
+
+      return null;
+    }
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
index 4e1ad83..c67c185 100644
--- a/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -19,7 +19,9 @@
 package org.apache.accumulo.test;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertSame;
@@ -43,7 +45,6 @@
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -882,7 +883,7 @@
       NewTableConfiguration ntc = new NewTableConfiguration().withSplits(nss("2", "4", "6"));
       client.tableOperations().create(tableName, ntc);
 
-      sleepUninterruptibly(2, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(2));
 
       int num = 100;
 
@@ -892,7 +893,7 @@
       byte[] e = new byte[0];
 
       for (int i = 0; i < num; i++) {
-        rows.add(FastFormat.toZeroPaddedString(abs(random.nextLong()), 16, 16, e));
+        rows.add(FastFormat.toZeroPaddedString(abs(RANDOM.get().nextLong()), 16, 16, e));
       }
 
       for (int i = 0; i < num; i++) {
@@ -967,7 +968,7 @@
       ColumnVisibility cvaob = new ColumnVisibility("A|B");
       ColumnVisibility cvaab = new ColumnVisibility("A&B");
 
-      switch (random.nextInt(3)) {
+      switch (RANDOM.get().nextInt(3)) {
         case 1:
           client.tableOperations().addSplits(tableName, nss("6"));
           break;
@@ -1196,19 +1197,19 @@
           new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY))) {
 
         for (int i = 0; i < 20; i++) {
-          int numRows = random.nextInt(10) + 1;
+          int numRows = RANDOM.get().nextInt(10) + 1;
 
           ArrayList<ByteSequence> changes = new ArrayList<>(numRows);
           ArrayList<ConditionalMutation> mutations = new ArrayList<>();
 
           for (int j = 0; j < numRows; j++) {
-            changes.add(rows.get(random.nextInt(rows.size())));
+            changes.add(rows.get(RANDOM.get().nextInt(rows.size())));
           }
 
           for (ByteSequence row : changes) {
             scanner.setRange(new Range(row.toString()));
             Stats stats = new Stats(scanner.iterator());
-            stats.set(random.nextInt(10), random.nextInt(Integer.MAX_VALUE));
+            stats.set(RANDOM.get().nextInt(10), RANDOM.get().nextInt(Integer.MAX_VALUE));
             mutations.add(stats.toMutation());
           }
 
@@ -1240,7 +1241,7 @@
 
       NewTableConfiguration ntc = new NewTableConfiguration();
 
-      switch (random.nextInt(3)) {
+      switch (RANDOM.get().nextInt(3)) {
         case 1:
           ntc = ntc.withSplits(nss("4"));
           break;
@@ -1257,7 +1258,7 @@
 
         for (int i = 0; i < 1000; i++) {
           rows.add(new ArrayByteSequence(
-              FastFormat.toZeroPaddedString(abs(random.nextLong()), 16, 16, new byte[0])));
+              FastFormat.toZeroPaddedString(abs(RANDOM.get().nextLong()), 16, 16, new byte[0])));
         }
 
         ArrayList<ConditionalMutation> mutations = new ArrayList<>();
@@ -1289,7 +1290,7 @@
         tp.shutdown();
 
         while (!tp.isTerminated()) {
-          tp.awaitTermination(1, TimeUnit.MINUTES);
+          tp.awaitTermination(1, MINUTES);
         }
 
         assertFalse(failed.get(), "A MutatorTask failed with an exception");
@@ -1385,7 +1386,7 @@
 
       try (
           ConditionalWriter cw = client.createConditionalWriter(table,
-              new ConditionalWriterConfig().setTimeout(3, TimeUnit.SECONDS));
+              new ConditionalWriterConfig().setTimeout(3, SECONDS));
           Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
 
         ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
diff --git a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
index 2480641..ba9bf2e 100644
--- a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
@@ -18,7 +18,9 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.net.URL;
@@ -31,7 +33,6 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -44,6 +45,7 @@
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -55,8 +57,6 @@
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
-import com.google.gson.Gson;
-
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
@@ -78,7 +78,7 @@
     URL url = new URL(uri + "/jmx");
     log.debug("Fetching web page " + url);
     String jsonString = FunctionalTestUtils.readWebPage(url).body();
-    Map<?,?> jsonObject = new Gson().fromJson(jsonString, Map.class);
+    Map<?,?> jsonObject = GSON.get().fromJson(jsonString, Map.class);
     List<?> beans = (List<?>) jsonObject.get("beans");
     for (Object bean : beans) {
       Map<?,?> map = (Map<?,?>) bean;
@@ -132,8 +132,10 @@
           fs.mkdirs(files);
           for (int i1 = 0; i1 < 100; i1++) {
             FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-                .forFile(files + "/bulk_" + i1 + "." + RFile.EXTENSION, fs, fs.getConf(),
-                    NoCryptoServiceFactory.NONE)
+                .forFile(
+                    UnreferencedTabletFile.of(fs,
+                        new Path(files + "/bulk_" + i1 + "." + RFile.EXTENSION)),
+                    fs, fs.getConf(), NoCryptoServiceFactory.NONE)
                 .withTableConfiguration(DefaultConfiguration.getInstance()).build();
             writer.startDefaultLocalityGroup();
             for (int j = 0x100; j < 0xfff; j += 3) {
@@ -162,10 +164,10 @@
         err.get();
       }
       es.shutdown();
-      es.awaitTermination(2, TimeUnit.MINUTES);
+      es.awaitTermination(2, MINUTES);
       log.info(
           String.format("Completed in %.2f seconds", (System.currentTimeMillis() - now) / 1000.));
-      sleepUninterruptibly(30, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(30));
       Map<?,?> map = getStats();
       map.forEach((k, v) -> {
         try {
diff --git a/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java b/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
index 29e483d..48d8ded 100644
--- a/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.minicluster.ServerType.TABLET_SERVER;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
@@ -28,13 +29,13 @@
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 
@@ -50,7 +51,8 @@
   public void test() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
       log.info("verifying that everything is up");
-      try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.forEach((k, v) -> {});
       }
       ManagerMonitorInfo stats = getStats(c);
@@ -61,22 +63,15 @@
       getCluster().killProcess(TABLET_SERVER,
           getCluster().getProcesses().get(TABLET_SERVER).iterator().next());
 
-      while (true) {
-        stats = getStats(c);
-        if (stats.tServerInfo.size() != 2) {
-          break;
-        }
-        UtilWaitThread.sleep(500);
-      }
+      Wait.waitFor(() -> getStats(c).tServerInfo.size() != 2, SECONDS.toMillis(60), 500);
+
+      stats = getStats(c);
       assertEquals(1, stats.tServerInfo.size());
       assertEquals(1, stats.badTServers.size() + stats.deadTabletServers.size());
-      while (true) {
-        stats = getStats(c);
-        if (!stats.deadTabletServers.isEmpty()) {
-          break;
-        }
-        UtilWaitThread.sleep(500);
-      }
+
+      Wait.waitFor(() -> !getStats(c).deadTabletServers.isEmpty(), SECONDS.toMillis(60), 500);
+
+      stats = getStats(c);
       assertEquals(1, stats.tServerInfo.size());
       assertEquals(0, stats.badTServers.size());
       assertEquals(1, stats.deadTabletServers.size());
diff --git a/test/src/main/java/org/apache/accumulo/test/DumpConfigIT.java b/test/src/main/java/org/apache/accumulo/test/DumpConfigIT.java
index 33e76ab..8727638 100644
--- a/test/src/main/java/org/apache/accumulo/test/DumpConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/DumpConfigIT.java
@@ -28,7 +28,7 @@
 import java.util.Collections;
 
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
@@ -67,13 +67,15 @@
     String site = FunctionalTestUtils.readAll(new FileInputStream(siteFileBackup));
     assertTrue(site.contains(Property.TABLE_FILE_BLOCK_SIZE.getKey()));
     assertTrue(site.contains("1234567"));
-    String meta = FunctionalTestUtils
-        .readAll(new FileInputStream(new File(folder, MetadataTable.NAME + ".cfg")));
+    String meta = FunctionalTestUtils.readAll(
+        new FileInputStream(new File(folder, AccumuloTable.METADATA.tableName() + ".cfg")));
     assertTrue(meta.contains(Property.TABLE_FILE_REPLICATION.getKey()));
     String systemPerm =
         FunctionalTestUtils.readAll(new FileInputStream(new File(folder, "root_user.cfg")));
     assertTrue(systemPerm.contains("grant System.ALTER_USER -s -u root"));
-    assertTrue(systemPerm.contains("grant Table.READ -t " + MetadataTable.NAME + " -u root"));
-    assertFalse(systemPerm.contains("grant Table.DROP -t " + MetadataTable.NAME + " -u root"));
+    assertTrue(systemPerm
+        .contains("grant Table.READ -t " + AccumuloTable.METADATA.tableName() + " -u root"));
+    assertFalse(systemPerm
+        .contains("grant Table.DROP -t " + AccumuloTable.METADATA.tableName() + " -u root"));
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
index c25140d..52827a4 100644
--- a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
@@ -43,8 +43,7 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-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.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
@@ -103,8 +102,8 @@
     }
 
     client.tableOperations().flush(table, null, null, true);
-    client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-    client.tableOperations().flush(RootTable.NAME, null, null, true);
+    client.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+    client.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
 
     Set<Entry<ServerType,Collection<ProcessReference>>> procs =
         getCluster().getProcesses().entrySet();
diff --git a/test/src/main/java/org/apache/accumulo/test/GCRunIT.java b/test/src/main/java/org/apache/accumulo/test/GCRunIT.java
index c57f4be..cab5ee6 100644
--- a/test/src/main/java/org/apache/accumulo/test/GCRunIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/GCRunIT.java
@@ -27,7 +27,7 @@
 import java.util.List;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -39,8 +39,7 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.gc.Reference;
-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.Ample;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.ColumnVisibility;
@@ -115,8 +114,8 @@
     fillMetadataEntries(table1, clone1);
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
 
-      client.securityOperations().grantTablePermission(getAdminPrincipal(), MetadataTable.NAME,
-          TablePermission.WRITE);
+      client.securityOperations().grantTablePermission(getAdminPrincipal(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
 
       String cloneId = client.tableOperations().tableIdMap().get(clone1);
 
@@ -126,7 +125,7 @@
       final Text colq = new Text(DIRECTORY_QUAL);
       m.putDelete(colf, colq, new ColumnVisibility());
 
-      try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME,
+      try (BatchWriter bw = client.createBatchWriter(AccumuloTable.METADATA.tableName(),
           new BatchWriterConfig().setMaxMemory(Math.max(m.estimatedMemoryUsed(), 1024))
               .setMaxWriteThreads(1).setTimeout(5_000, TimeUnit.MILLISECONDS))) {
         log.info("forcing delete of srv:dir with mutation {}", m.prettyPrint());
@@ -155,16 +154,11 @@
   public void forceMissingPrevRowTest() {}
 
   private void scanReferences(GCRun userGC) {
-    final AtomicInteger counter = new AtomicInteger(0);
     // loop through the user table references - the row deleted above should violate dir present.
-    var userTableIter = userGC.getReferences().iterator();
-    while (userTableIter.hasNext()) {
-      Reference ref = userTableIter.next();
-      counter.incrementAndGet();
-      log.trace("user ref: {}", ref);
+    try (Stream<Reference> references = userGC.getReferences()) {
+      long count = references.peek(ref -> log.trace("user ref: {}", ref)).count();
+      assertTrue(count > 0);
     }
-
-    assertTrue(counter.get() > 0);
   }
 
   private void fillMetadataEntries(final String table1, final String clone1) throws Exception {
@@ -193,8 +187,8 @@
       client.tableOperations().compact(table1, new CompactionConfig().setWait(true));
       client.tableOperations().delete(table1);
 
-      client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-      client.tableOperations().flush(RootTable.NAME, null, null, true);
+      client.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+      client.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
 
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java b/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java
index 2fe0862..67f1282 100644
--- a/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
@@ -26,13 +27,13 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -69,11 +70,11 @@
       cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
       cluster.getClusterControl().start(ServerType.GARBAGE_COLLECTOR);
       cluster.getClusterControl().start(ServerType.TABLET_SERVER);
-      try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.forEach((k, v) -> {});
       }
-      UtilWaitThread.sleep(3 * 5_000);
-      assertEquals(2, countWALsInFS(cluster));
+      Wait.waitFor(() -> countWALsInFS(cluster) == 2, SECONDS.toMillis(120), SECONDS.toMillis(15));
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java b/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java
index e3769f7..b63f555 100644
--- a/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java
+++ b/test/src/main/java/org/apache/accumulo/test/GenerateSequentialRFile.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -58,10 +59,10 @@
   public void run() {
     try {
       final Configuration conf = new Configuration();
-      Path p = new Path(opts.filePath);
-      final FileSystem fs = p.getFileSystem(conf);
+      final FileSystem fs = FileSystem.get(conf);
+      UnreferencedTabletFile file = UnreferencedTabletFile.of(fs, new Path(opts.filePath));
       FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-          .forFile(opts.filePath, fs, conf, NoCryptoServiceFactory.NONE)
+          .forFile(file, fs, conf, NoCryptoServiceFactory.NONE)
           .withTableConfiguration(DefaultConfiguration.getInstance()).build();
 
       writer.startDefaultLocalityGroup();
diff --git a/test/src/main/java/org/apache/accumulo/test/GetManagerStats.java b/test/src/main/java/org/apache/accumulo/test/GetManagerStats.java
index 6727371..9101f3f 100644
--- a/test/src/main/java/org/apache/accumulo/test/GetManagerStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/GetManagerStats.java
@@ -23,12 +23,12 @@
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.manager.thrift.BulkImportStatus;
 import org.apache.accumulo.core.manager.thrift.DeadServer;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.BulkImportStatus;
-import org.apache.accumulo.core.master.thrift.RecoveryStatus;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.RecoveryStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.server.ServerContext;
diff --git a/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java b/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java
index 49f01c1..4fc909c 100644
--- a/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ImportExportIT.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.test;
 
 import static org.apache.accumulo.core.Constants.IMPORT_MAPPINGS_FILE;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -27,11 +28,14 @@
 import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.nio.file.Paths;
 import java.time.Duration;
 import java.util.Arrays;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map.Entry;
 import java.util.Set;
 
@@ -45,22 +49,29 @@
 import org.apache.accumulo.core.client.admin.ImportConfiguration;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.manager.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.test.util.FileMetadataUtil;
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -82,8 +93,9 @@
     return Duration.ofMinutes(1);
   }
 
-  @Test
-  public void testExportImportThenScan() throws Exception {
+  @ParameterizedTest
+  @ValueSource(booleans = {true, false})
+  public void testExportImportThenScan(boolean fenced) throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
 
       String[] tableNames = getUniqueNames(2);
@@ -92,7 +104,7 @@
 
       try (BatchWriter bw = client.createBatchWriter(srcTable)) {
         for (int row = 0; row < 1000; row++) {
-          Mutation m = new Mutation(Integer.toString(row));
+          Mutation m = new Mutation("row_" + String.format("%010d", row));
           for (int col = 0; col < 100; col++) {
             m.put(Integer.toString(col), "", Integer.toString(col * 2));
           }
@@ -102,6 +114,14 @@
 
       client.tableOperations().compact(srcTable, null, null, true, true);
 
+      int expected = 100000;
+      // Test that files with ranges and are fenced work with export/import
+      if (fenced) {
+        // Split file into 3 ranges of 10000, 20000, and 5000 for a total of 35000
+        FileMetadataUtil.splitFilesIntoRanges(getServerContext(), srcTable, createRanges());
+        expected = 35000;
+      }
+
       // Make a directory we can use to throw the export and import directories
       // Must exist on the filesystem the cluster is running.
       FileSystem fs = cluster.getFileSystem();
@@ -153,7 +173,7 @@
       while ((line = reader.readLine()) != null) {
         Path p = new Path(line.substring(5));
         assertTrue(fs.exists(p), "File doesn't exist: " + p);
-        Path importDir = importDirAry[random.nextInt(importDirAry.length)];
+        Path importDir = importDirAry[RANDOM.get().nextInt(importDirAry.length)];
         Path dest = new Path(importDir, p.getName());
         assertFalse(fs.exists(dest), "Did not expect " + dest + " to exist");
         FileUtil.copy(fs, p, fs, dest, false, fs.getConf());
@@ -174,7 +194,8 @@
       // Get all `file` colfams from the metadata table for the new table
       log.info("Imported into table with ID: {}", tableId);
 
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s =
+          client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.setRange(TabletsSection.getRange(TableId.of(tableId)));
         s.fetchColumnFamily(DataFileColumnFamily.NAME);
         ServerColumnFamily.DIRECTORY_COLUMN.fetch(s);
@@ -186,9 +207,12 @@
           if (k.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
             // The file should be an absolute URI (file:///...), not a relative path
             // (/b-000.../I000001.rf)
-            String fileUri = k.getColumnQualifier().toString();
-            assertFalse(looksLikeRelativePath(fileUri),
-                "Imported files should have absolute URIs, not relative: " + fileUri);
+            var tabFile = StoredTabletFile.of(k.getColumnQualifier());
+            // Verify that the range is set correctly on the StoredTabletFile
+            assertEquals(fenced, !tabFile.getRange().isInfiniteStartKey()
+                || !tabFile.getRange().isInfiniteStopKey());
+            assertFalse(looksLikeRelativePath(tabFile.getMetadataPath()),
+                "Imported files should have absolute URIs, not relative: " + tabFile);
           } else if (k.getColumnFamily().equals(ServerColumnFamily.NAME)) {
             assertFalse(looksLikeRelativePath(value),
                 "Server directory should have absolute URI, not relative: " + value);
@@ -201,12 +225,13 @@
       // Online the original table before we verify equivalence
       client.tableOperations().online(srcTable, true);
 
-      verifyTableEquality(client, srcTable, destTable);
+      verifyTableEquality(client, srcTable, destTable, expected);
     }
   }
 
-  @Test
-  public void testExportImportOffline() throws Exception {
+  @ParameterizedTest
+  @ValueSource(booleans = {true, false})
+  public void testExportImportOffline(boolean fenced) throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
 
       String[] tableNames = getUniqueNames(2);
@@ -215,7 +240,7 @@
 
       try (BatchWriter bw = client.createBatchWriter(srcTable)) {
         for (int row = 0; row < 1000; row++) {
-          Mutation m = new Mutation(Integer.toString(row));
+          Mutation m = new Mutation("row_" + String.format("%010d", row));
           for (int col = 0; col < 100; col++) {
             m.put(Integer.toString(col), "", Integer.toString(col * 2));
           }
@@ -225,6 +250,14 @@
 
       client.tableOperations().compact(srcTable, new CompactionConfig());
 
+      int expected = 100000;
+      // Test that files with ranges and are fenced work with export/import
+      if (fenced) {
+        // Split file into 3 ranges of 10000, 20000, and 5000 for a total of 35000
+        FileMetadataUtil.splitFilesIntoRanges(getServerContext(), srcTable, createRanges());
+        expected = 35000;
+      }
+
       // Make export and import directories
       FileSystem fs = cluster.getFileSystem();
       log.info("Using FileSystem: " + fs);
@@ -271,7 +304,7 @@
       while ((line = reader.readLine()) != null) {
         Path p = new Path(line.substring(5));
         assertTrue(fs.exists(p), "File doesn't exist: " + p);
-        Path importDir = importDirAry[random.nextInt(importDirAry.length)];
+        Path importDir = importDirAry[RANDOM.get().nextInt(importDirAry.length)];
         Path dest = new Path(importDir, p.getName());
         assertFalse(fs.exists(dest), "Did not expect " + dest + " to exist");
         FileUtil.copy(fs, p, fs, dest, false, fs.getConf());
@@ -299,7 +332,8 @@
       client.tableOperations().online(destTable, true);
 
       // Get all `file` colfams from the metadata table for the new table
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s =
+          client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.setRange(TabletsSection.getRange(TableId.of(tableId)));
         s.fetchColumnFamily(DataFileColumnFamily.NAME);
         ServerColumnFamily.DIRECTORY_COLUMN.fetch(s);
@@ -310,9 +344,13 @@
           String value = fileEntry.getValue().toString();
           if (k.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
             // file should be an absolute URI (file:///...), not relative (/b-000.../I000001.rf)
-            String fileUri = k.getColumnQualifier().toString();
-            assertFalse(looksLikeRelativePath(fileUri),
-                "Imported files should have absolute URIs, not relative: " + fileUri);
+            var tabFile = StoredTabletFile.of(k.getColumnQualifier());
+            // Verify that the range is set correctly on the StoredTabletFile
+            assertEquals(fenced, !tabFile.getRange().isInfiniteStartKey()
+                || !tabFile.getRange().isInfiniteStopKey());
+            assertFalse(looksLikeRelativePath(tabFile.getMetadataPath()),
+                "Imported files should have absolute URIs, not relative: "
+                    + tabFile.getMetadataPath());
           } else if (k.getColumnFamily().equals(ServerColumnFamily.NAME)) {
             assertFalse(looksLikeRelativePath(value),
                 "Server directory should have absolute URI, not relative: " + value);
@@ -324,7 +362,7 @@
       // Online the original table before we verify equivalence
       client.tableOperations().online(srcTable, true);
 
-      verifyTableEquality(client, srcTable, destTable);
+      verifyTableEquality(client, srcTable, destTable, expected);
       assertTrue(verifyMappingsFile(tableId), "Did not find mappings file");
     }
   }
@@ -348,20 +386,70 @@
     return false;
   }
 
-  private void verifyTableEquality(AccumuloClient client, String srcTable, String destTable)
-      throws Exception {
+  /**
+   * Validate that files exported with Accumulo 2.x without fence ranges can be imported into
+   * version that require the fenced ranges (3.1 and later)
+   */
+  @Test
+  public void importV2data() throws Exception {
+    final String dataRoot = "./target/classes/v2_import_test";
+    final String dataSrc = dataRoot + "/data";
+    final String importDir = dataRoot + "/import";
+
+    // copy files each run will "move the files" on import, allows multiple runs in IDE without
+    // rebuild
+    java.nio.file.Path importDirPath = Paths.get(importDir);
+    java.nio.file.Files.createDirectories(importDirPath);
+    FileUtils.copyDirectory(new File(dataSrc), new File(importDir));
+
+    String table = getUniqueNames(1)[0];
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      log.debug("importing from: {} into table: {}", importDir, table);
+      client.tableOperations().importTable(table, importDir);
+
+      int rowCount = 0;
+      try (Scanner s = client.createScanner(table, Authorizations.EMPTY)) {
+        for (Entry<Key,Value> entry : s) {
+          log.trace("data:{}", entry);
+          rowCount++;
+        }
+      }
+      assertEquals(7, rowCount);
+      int metaFileCount = 0;
+      try (Scanner s =
+          client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
+        TableId tid = TableId.of(client.tableOperations().tableIdMap().get(table));
+        s.setRange(TabletsSection.getRange(tid));
+        s.fetchColumnFamily(DataFileColumnFamily.NAME);
+        for (Entry<Key,Value> entry : s) {
+          log.trace("metadata file:{}", entry);
+          metaFileCount++;
+        }
+      }
+      final List<Text> expectedSplits = List.of(new Text("2"), new Text("4"), new Text("6"));
+      assertEquals(expectedSplits, client.tableOperations().listSplits(table));
+      assertEquals(4, metaFileCount);
+    }
+  }
+
+  private void verifyTableEquality(AccumuloClient client, String srcTable, String destTable,
+      int expected) throws Exception {
     Iterator<Entry<Key,Value>> src =
         client.createScanner(srcTable, Authorizations.EMPTY).iterator(),
         dest = client.createScanner(destTable, Authorizations.EMPTY).iterator();
     assertTrue(src.hasNext(), "Could not read any data from source table");
     assertTrue(dest.hasNext(), "Could not read any data from destination table");
+    int entries = 0;
     while (src.hasNext() && dest.hasNext()) {
       Entry<Key,Value> orig = src.next(), copy = dest.next();
       assertEquals(orig.getKey(), copy.getKey());
       assertEquals(orig.getValue(), copy.getValue());
+      entries++;
     }
     assertFalse(src.hasNext(), "Source table had more data to read");
     assertFalse(dest.hasNext(), "Dest table had more data to read");
+    assertEquals(expected, entries);
   }
 
   private boolean looksLikeRelativePath(String uri) {
@@ -371,4 +459,12 @@
       return uri.startsWith("/" + Constants.CLONE_PREFIX);
     }
   }
+
+  private Set<Range> createRanges() {
+    // Split file into ranges of 10000, 20000, and 5000 for a total of 35000
+    return Set.of(
+        new Range("row_" + String.format("%010d", 100), "row_" + String.format("%010d", 199)),
+        new Range("row_" + String.format("%010d", 300), "row_" + String.format("%010d", 499)),
+        new Range("row_" + String.format("%010d", 700), "row_" + String.format("%010d", 749)));
+  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java b/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java
index 05cc7a3..ea7432b 100644
--- a/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/IteratorEnvIT.java
@@ -43,7 +43,6 @@
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.WrappingIterator;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
@@ -149,22 +148,10 @@
       IteratorEnvironment env) {
     TableId expectedTableId = TableId.of(opts.get("expected.table.id"));
 
-    // verify getServiceEnv() and getPluginEnv() are the same objects,
-    // so further checks only need to use getPluginEnv()
-    @SuppressWarnings("deprecation")
-    ServiceEnvironment serviceEnv = env.getServiceEnv();
     PluginEnvironment pluginEnv = env.getPluginEnv();
-    if (serviceEnv != pluginEnv) {
-      throw new RuntimeException("Test failed - assertSame(getServiceEnv(),getPluginEnv())");
-    }
 
-    // verify property exists on the table config (deprecated and new),
+    // verify property exists on the table config,
     // with and without custom prefix, but not in the system config
-    @SuppressWarnings("deprecation")
-    String accTableConf = env.getConfig().get("table.custom.iterator.env.test");
-    if (!"value1".equals(accTableConf)) {
-      throw new RuntimeException("Test failed - Expected table property not found in getConfig().");
-    }
     var tableConf = pluginEnv.getConfiguration(env.getTableId());
     if (!"value1".equals(tableConf.get("table.custom.iterator.env.test"))) {
       throw new RuntimeException("Test failed - Expected table property not found in table conf.");
diff --git a/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java
index dcaced8..793b9aa 100644
--- a/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.test;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 import java.time.Duration;
 import java.util.HashSet;
@@ -32,16 +32,13 @@
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
-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.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.ServerContext;
@@ -75,9 +72,10 @@
       ClientContext context = (ClientContext) c;
       ServerContext serverContext = cluster.getServerContext();
       String table = this.getUniqueNames(1)[0];
-      c.securityOperations().grantTablePermission("root", MetadataTable.NAME,
+      c.securityOperations().grantTablePermission("root", AccumuloTable.METADATA.tableName(),
           TablePermission.WRITE);
-      c.securityOperations().grantTablePermission("root", RootTable.NAME, TablePermission.WRITE);
+      c.securityOperations().grantTablePermission("root", AccumuloTable.ROOT.tableName(),
+          TablePermission.WRITE);
       SortedSet<Text> partitions = new TreeSet<>();
       for (String part : "a b c d e f g h i j k l m n o p q r s t u v w x y z".split(" ")) {
         partitions.add(new Text(part));
@@ -85,11 +83,12 @@
       NewTableConfiguration ntc = new NewTableConfiguration().withSplits(partitions);
       c.tableOperations().create(table, ntc);
       // scan the metadata table and get the two table location states
-      Set<TabletMetadata.Location> states = new HashSet<>();
-      Set<TabletLocationState> oldLocations = new HashSet<>();
-      TabletStateStore store = TabletStateStore.getStoreForLevel(DataLevel.USER, context);
+      final Set<TabletMetadata.Location> states = new HashSet<>();
+      final Set<TabletLocationState> oldLocations = new HashSet<>();
+      final TabletStateStore store = TabletStateStore.getStoreForLevel(DataLevel.USER, context);
+
       while (states.size() < 2) {
-        UtilWaitThread.sleep(250);
+        Thread.sleep(250);
         oldLocations.clear();
         for (TabletLocationState tls : store) {
           if (tls.current != null) {
@@ -102,20 +101,14 @@
       // Kill a tablet server... we don't care which one... wait for everything to be reassigned
       cluster.killProcess(ServerType.TABLET_SERVER,
           cluster.getProcesses().get(ServerType.TABLET_SERVER).iterator().next());
-      Set<TabletMetadata.Location> replStates = new HashSet<>();
-      @SuppressWarnings("deprecation")
-      TableId repTable = org.apache.accumulo.core.replication.ReplicationTable.ID;
       // Find out which tablet server remains
       while (true) {
-        UtilWaitThread.sleep(1000);
+        Thread.sleep(1000);
         states.clear();
-        replStates.clear();
         boolean allAssigned = true;
         for (TabletLocationState tls : store) {
           if (tls != null && tls.current != null) {
             states.add(tls.current);
-          } else if (tls != null && tls.extent.equals(new KeyExtent(repTable, null, null))) {
-            replStates.add(tls.current);
           } else {
             allAssigned = false;
           }
@@ -125,7 +118,6 @@
           break;
         }
       }
-      assertEquals(1, replStates.size());
       assertEquals(1, states.size());
       // pick an assigned tablet and assign it to the old tablet
       TabletLocationState moved = null;
@@ -134,7 +126,7 @@
           moved = old;
         }
       }
-      assertNotEquals(null, moved);
+      assertNotNull(moved);
       // throw a mutation in as if we were the dying tablet
       TabletMutator tabletMutator = serverContext.getAmple().mutateTablet(moved.extent);
       tabletMutator.putLocation(moved.current);
@@ -142,21 +134,21 @@
       // wait for the manager to fix the problem
       waitForCleanStore(store);
       // now jam up the metadata table
-      tabletMutator =
-          serverContext.getAmple().mutateTablet(new KeyExtent(MetadataTable.ID, null, null));
+      tabletMutator = serverContext.getAmple()
+          .mutateTablet(new KeyExtent(AccumuloTable.METADATA.tableId(), null, null));
       tabletMutator.putLocation(moved.current);
       tabletMutator.mutate();
       waitForCleanStore(TabletStateStore.getStoreForLevel(DataLevel.METADATA, context));
     }
   }
 
-  private void waitForCleanStore(TabletStateStore store) {
+  private void waitForCleanStore(TabletStateStore store) throws InterruptedException {
     while (true) {
       try (ClosableIterator<TabletLocationState> iter = store.iterator()) {
         iter.forEachRemaining(t -> {});
       } catch (Exception ex) {
-        System.out.println(ex);
-        UtilWaitThread.sleep(250);
+        System.out.println(ex.getMessage());
+        Thread.sleep(250);
         continue;
       }
       break;
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java b/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java
index 2bbc921..2bf338b 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaConstraintRetryIT.java
@@ -30,7 +30,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.server.ServerContext;
@@ -48,8 +48,8 @@
   @Test
   public void test() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      client.securityOperations().grantTablePermission(getAdminPrincipal(), MetadataTable.NAME,
-          TablePermission.WRITE);
+      client.securityOperations().grantTablePermission(getAdminPrincipal(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
 
       ServerContext context = getServerContext();
       KeyExtent extent = new KeyExtent(TableId.of("5"), null, null);
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
index 64df9c7..2608db3 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaGetsReadersIT.java
@@ -18,14 +18,13 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -37,7 +36,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
@@ -87,7 +86,7 @@
       try (BatchWriter bw = c.createBatchWriter(tableName)) {
         for (int i = 0; i < 50000; i++) {
           byte[] row = new byte[100];
-          random.nextBytes(row);
+          RANDOM.get().nextBytes(row);
           Mutation m = new Mutation(row);
           m.put("", "", "");
           bw.addMutation(m);
@@ -99,10 +98,10 @@
       t1.start();
       Thread t2 = slowScan(c, tableName, stop);
       t2.start();
-      sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+      Thread.sleep(500);
       long now = System.currentTimeMillis();
 
-      try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.forEach((k, v) -> {});
       }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java
index 11e697e..bf8e98c 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java
@@ -32,8 +32,7 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
-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.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
@@ -79,14 +78,14 @@
         log.info("Data written to table {}", i);
         i++;
       }
-      c.tableOperations().flush(MetadataTable.NAME, null, null, true);
-      c.tableOperations().flush(RootTable.NAME, null, null, true);
+      c.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+      c.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
       SortedSet<Text> splits = new TreeSet<>();
       for (i = 1; i < tables.length; i++) {
         splits.add(new Text("" + i));
       }
-      c.tableOperations().addSplits(MetadataTable.NAME, splits);
-      log.info("Added {} splits to {}", splits.size(), MetadataTable.NAME);
+      c.tableOperations().addSplits(AccumuloTable.METADATA.tableName(), splits);
+      log.info("Added {} splits to {}", splits.size(), AccumuloTable.METADATA.tableName());
       c.instanceOperations().waitForBalance();
       log.info("Restarting");
       getCluster().getClusterControl().kill(ServerType.TABLET_SERVER, "localhost");
diff --git a/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java b/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java
index 72b63d2..2133e4d 100644
--- a/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MetaSplitIT.java
@@ -18,24 +18,35 @@
  */
 package org.apache.accumulo.test;
 
+import static org.apache.accumulo.test.util.FileMetadataUtil.countFencedFiles;
+import static org.apache.accumulo.test.util.FileMetadataUtil.verifyMergedMarkerCleared;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.Accumulo;
 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.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.conf.Property;
+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.schema.Ample;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.AfterEach;
@@ -58,14 +69,15 @@
   public void saveMetadataSplits() throws Exception {
     if (getClusterType() == ClusterType.STANDALONE) {
       try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-        Collection<Text> splits = client.tableOperations().listSplits(MetadataTable.NAME);
+        Collection<Text> splits =
+            client.tableOperations().listSplits(AccumuloTable.METADATA.tableName());
         // We expect a single split
         if (!splits.equals(Arrays.asList(new Text("~")))) {
           log.info("Existing splits on metadata table. Saving them, and applying"
               + " single original split of '~'");
           metadataSplits = splits;
-          client.tableOperations().merge(MetadataTable.NAME, null, null);
-          client.tableOperations().addSplits(MetadataTable.NAME,
+          client.tableOperations().merge(AccumuloTable.METADATA.tableName(), null, null);
+          client.tableOperations().addSplits(AccumuloTable.METADATA.tableName(),
               new TreeSet<>(Collections.singleton(new Text("~"))));
         }
       }
@@ -77,8 +89,9 @@
     if (metadataSplits != null) {
       log.info("Restoring split on metadata table");
       try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-        client.tableOperations().merge(MetadataTable.NAME, null, null);
-        client.tableOperations().addSplits(MetadataTable.NAME, new TreeSet<>(metadataSplits));
+        client.tableOperations().merge(AccumuloTable.METADATA.tableName(), null, null);
+        client.tableOperations().addSplits(AccumuloTable.METADATA.tableName(),
+            new TreeSet<>(metadataSplits));
       }
     }
   }
@@ -89,14 +102,14 @@
       SortedSet<Text> splits = new TreeSet<>();
       splits.add(new Text("5"));
       assertThrows(AccumuloException.class,
-          () -> client.tableOperations().addSplits(RootTable.NAME, splits));
+          () -> client.tableOperations().addSplits(AccumuloTable.ROOT.tableName(), splits));
     }
   }
 
   @Test
   public void testRootTableMerge() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      client.tableOperations().merge(RootTable.NAME, null, null);
+      client.tableOperations().merge(AccumuloTable.ROOT.tableName(), null, null);
     }
   }
 
@@ -105,29 +118,63 @@
     for (String point : points) {
       splits.add(new Text(point));
     }
-    opts.addSplits(MetadataTable.NAME, splits);
+    opts.addSplits(AccumuloTable.METADATA.tableName(), splits);
   }
 
   @Test
   public void testMetadataTableSplit() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      // disable compactions
+      client.tableOperations().setProperty(AccumuloTable.METADATA.tableName(),
+          Property.TABLE_MAJC_RATIO.getKey(), "9999");
+
       TableOperations opts = client.tableOperations();
       for (int i = 1; i <= 10; i++) {
         opts.create(Integer.toString(i));
       }
       try {
-        opts.merge(MetadataTable.NAME, new Text("01"), new Text("02"));
+        assertEquals(0, countFencedFiles(getServerContext(), AccumuloTable.METADATA.tableName()));
+        verifyMetadataTableScan(client);
+        opts.merge(AccumuloTable.METADATA.tableName(), new Text("01"), new Text("02"));
         checkMetadataSplits(1, opts);
+        verifyMetadataTableScan(client);
         addSplits(opts, "4 5 6 7 8".split(" "));
         checkMetadataSplits(6, opts);
-        opts.merge(MetadataTable.NAME, new Text("6"), new Text("9"));
+        verifyMetadataTableScan(client);
+
+        opts.merge(AccumuloTable.METADATA.tableName(), new Text("6"), new Text("9"));
         checkMetadataSplits(4, opts);
+        // Merging tablets should produce fenced files because of no-chop merge
+        assertTrue(countFencedFiles(getServerContext(), AccumuloTable.METADATA.tableName()) > 0);
+        verifyMetadataTableScan(client);
+        // Verify that the MERGED marker was cleared and doesn't exist on any tablet
+        verifyMergedMarkerCleared(getServerContext(), AccumuloTable.METADATA.tableId());
+
         addSplits(opts, "44 55 66 77 88".split(" "));
         checkMetadataSplits(9, opts);
-        opts.merge(MetadataTable.NAME, new Text("5"), new Text("7"));
+        assertTrue(countFencedFiles(getServerContext(), AccumuloTable.METADATA.tableName()) > 0);
+        verifyMetadataTableScan(client);
+        // Verify that the MERGED marker was cleared and doesn't exist on any tablet
+        verifyMergedMarkerCleared(getServerContext(), AccumuloTable.METADATA.tableId());
+
+        opts.merge(AccumuloTable.METADATA.tableName(), new Text("5"), new Text("7"));
         checkMetadataSplits(6, opts);
-        opts.merge(MetadataTable.NAME, null, null);
+        assertTrue(countFencedFiles(getServerContext(), AccumuloTable.METADATA.tableName()) > 0);
+        verifyMetadataTableScan(client);
+        // Verify that the MERGED marker was cleared and doesn't exist on any tablet
+        verifyMergedMarkerCleared(getServerContext(), AccumuloTable.METADATA.tableId());
+
+        opts.merge(AccumuloTable.METADATA.tableName(), null, null);
         checkMetadataSplits(0, opts);
+        assertTrue(countFencedFiles(getServerContext(), AccumuloTable.METADATA.tableName()) > 0);
+        verifyMetadataTableScan(client);
+        // Verify that the MERGED marker was cleared and doesn't exist on any tablet
+        verifyMergedMarkerCleared(getServerContext(), AccumuloTable.METADATA.tableId());
+
+        opts.compact(AccumuloTable.METADATA.tableName(), new CompactionConfig());
+        // Should be no more fenced files after compaction
+        assertEquals(0, countFencedFiles(getServerContext(), AccumuloTable.METADATA.tableName()));
+        verifyMetadataTableScan(client);
       } finally {
         for (int i = 1; i <= 10; i++) {
           opts.delete(Integer.toString(i));
@@ -136,16 +183,42 @@
     }
   }
 
+  // Count the number of entries that can be read in the Metadata table
+  // This verifies all the entries can still be read after splits/merges
+  // when ranged files are used
+  private void verifyMetadataTableScan(AccumuloClient client) throws Exception {
+    var tables = client.tableOperations().tableIdMap();
+    var expectedExtents = tables.entrySet().stream()
+        .filter(e -> !e.getKey().startsWith("accumulo.")).map(Map.Entry::getValue).map(TableId::of)
+        .map(tid -> new KeyExtent(tid, null, null)).collect(Collectors.toSet());
+    // Verify we have 10 tablets for metadata
+    assertEquals(10, expectedExtents.size());
+
+    // Scan each tablet to verify data exists
+    var ample = ((ClientContext) client).getAmple();
+    try (var tablets = ample.readTablets().forLevel(Ample.DataLevel.USER).build()) {
+      for (var tablet : tablets) {
+        assertTrue(expectedExtents.remove(tablet.getExtent()));
+        // check a few fields that should always be present in tablet metadata
+        assertNotNull(tablet.getDirName());
+        assertNotNull(tablet.getTime());
+      }
+    }
+
+    // ensure all expected extents were seen
+    assertEquals(0, expectedExtents.size());
+  }
+
   private static void checkMetadataSplits(int numSplits, TableOperations opts)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       InterruptedException {
     for (int i = 0; i < 10; i++) {
-      if (opts.listSplits(MetadataTable.NAME).size() == numSplits) {
+      if (opts.listSplits(AccumuloTable.METADATA.tableName()).size() == numSplits) {
         break;
       }
       Thread.sleep(2000);
     }
-    Collection<Text> splits = opts.listSplits(MetadataTable.NAME);
+    Collection<Text> splits = opts.listSplits(AccumuloTable.METADATA.tableName());
     assertEquals(numSplits, splits.size(), "Actual metadata table splits: " + splits);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index 87e931b..f2d8c4e 100644
--- a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -35,8 +35,7 @@
 import org.apache.accumulo.core.conf.Property;
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
@@ -83,9 +82,9 @@
   public void setupMetadataPermission() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
       rootHasWritePermission = client.securityOperations().hasTablePermission("root",
-          MetadataTable.NAME, TablePermission.WRITE);
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
       if (!rootHasWritePermission) {
-        client.securityOperations().grantTablePermission("root", MetadataTable.NAME,
+        client.securityOperations().grantTablePermission("root", AccumuloTable.METADATA.tableName(),
             TablePermission.WRITE);
         // Make sure it propagates through ZK
         Thread.sleep(5000);
@@ -98,15 +97,15 @@
     try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
       // Final state doesn't match the original
       if (rootHasWritePermission != client.securityOperations().hasTablePermission("root",
-          MetadataTable.NAME, TablePermission.WRITE)) {
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE)) {
         if (rootHasWritePermission) {
           // root had write permission when starting, ensure root still does
-          client.securityOperations().grantTablePermission("root", MetadataTable.NAME,
-              TablePermission.WRITE);
+          client.securityOperations().grantTablePermission("root",
+              AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
         } else {
           // root did not have write permission when starting, ensure that it does not
-          client.securityOperations().revokeTablePermission("root", MetadataTable.NAME,
-              TablePermission.WRITE);
+          client.securityOperations().revokeTablePermission("root",
+              AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
         }
       }
     }
@@ -129,8 +128,9 @@
 
       fs.create(new Path(emptyWalog.toURI())).close();
 
-      assertTrue(client.securityOperations().hasTablePermission("root", MetadataTable.NAME,
-          TablePermission.WRITE), "root user did not have write permission to metadata table");
+      assertTrue(client.securityOperations().hasTablePermission("root",
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE),
+          "root user did not have write permission to metadata table");
 
       String tableName = getUniqueNames(1)[0];
       client.tableOperations().create(tableName);
@@ -138,8 +138,7 @@
       TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
       assertNotNull(tableId, "Table ID was null");
 
-      LogEntry logEntry =
-          new LogEntry(new KeyExtent(tableId, null, null), 0, emptyWalog.toURI().toString());
+      LogEntry logEntry = LogEntry.fromPath(emptyWalog.toURI().toString());
 
       log.info("Taking {} offline", tableName);
       client.tableOperations().offline(tableName, true);
@@ -148,9 +147,9 @@
 
       Text row = TabletsSection.encodeRow(tableId, null);
       Mutation m = new Mutation(row);
-      m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
+      logEntry.addToMutation(m);
 
-      try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
+      try (BatchWriter bw = client.createBatchWriter(AccumuloTable.METADATA.tableName())) {
         bw.addMutation(m);
       }
 
@@ -188,8 +187,9 @@
           DfsLogger.LOG_FILE_HEADER_V4.length() / 2);
       wal.close();
 
-      assertTrue(client.securityOperations().hasTablePermission("root", MetadataTable.NAME,
-          TablePermission.WRITE), "root user did not have write permission to metadata table");
+      assertTrue(client.securityOperations().hasTablePermission("root",
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE),
+          "root user did not have write permission to metadata table");
 
       String tableName = getUniqueNames(1)[0];
       client.tableOperations().create(tableName);
@@ -197,7 +197,7 @@
       TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
       assertNotNull(tableId, "Table ID was null");
 
-      LogEntry logEntry = new LogEntry(null, 0, partialHeaderWalog.toURI().toString());
+      LogEntry logEntry = LogEntry.fromPath(partialHeaderWalog.toURI().toString());
 
       log.info("Taking {} offline", tableName);
       client.tableOperations().offline(tableName, true);
@@ -206,9 +206,9 @@
 
       Text row = TabletsSection.encodeRow(tableId, null);
       Mutation m = new Mutation(row);
-      m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
+      logEntry.addToMutation(m);
 
-      try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
+      try (BatchWriter bw = client.createBatchWriter(AccumuloTable.METADATA.tableName())) {
         bw.addMutation(m);
       }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
index 67dd4ad..a263883 100644
--- a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java
@@ -18,14 +18,14 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.IOException;
 import java.time.Duration;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -37,7 +37,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -85,7 +85,7 @@
       System.out.println("writing");
       for (i = 0; i < 1_000_000; i++) {
         // make non-negative avoiding Math.abs, because that can still be negative
-        long randomRow = random.nextLong() & Long.MAX_VALUE;
+        long randomRow = RANDOM.get().nextLong() & Long.MAX_VALUE;
         assertTrue(randomRow >= 0);
         final int table = (int) (randomRow % N);
         final Mutation m = new Mutation(Long.toHexString(randomRow));
@@ -125,12 +125,13 @@
       try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
         int i = 0;
         while (!stop.get()) {
-          sleepUninterruptibly(10, TimeUnit.SECONDS);
+          Thread.sleep(SECONDS.toMillis(10));
           System.out.println("Restarting");
           getCluster().getClusterControl().stop(ServerType.TABLET_SERVER);
           getCluster().start();
           // read the metadata table to know everything is back up
-          try (Scanner scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+          try (Scanner scanner =
+              client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
             scanner.forEach((k, v) -> {});
           }
           i++;
diff --git a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
index 4cc8e06..3665c48 100644
--- a/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/NamespacesIT.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -39,7 +39,6 @@
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterUser;
 import org.apache.accumulo.core.client.Accumulo;
@@ -74,8 +73,7 @@
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
-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.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
@@ -136,7 +134,7 @@
         c.tableOperations().delete(t);
       }
     }
-    assertEquals(3, c.tableOperations().list().size());
+    assertEquals(2, c.tableOperations().list().size());
     for (String n : c.namespaceOperations().list()) {
       if (!n.equals(Namespace.ACCUMULO.name()) && !n.equals(Namespace.DEFAULT.name())) {
         c.namespaceOperations().delete(n);
@@ -403,7 +401,7 @@
       c.namespaceOperations().checkIteratorConflicts(namespace, setting,
           EnumSet.allOf(IteratorScope.class));
       c.namespaceOperations().attachIterator(namespace, setting);
-      sleepUninterruptibly(2, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(2));
       var e = assertThrows(AccumuloException.class, () -> c.namespaceOperations()
           .checkIteratorConflicts(namespace, setting, EnumSet.allOf(IteratorScope.class)));
       assertEquals(IllegalArgumentException.class, e.getCause().getClass());
@@ -420,7 +418,7 @@
       // verify can see inserted entry again
       c.namespaceOperations().removeIterator(namespace, setting.getName(),
           EnumSet.allOf(IteratorScope.class));
-      sleepUninterruptibly(2, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(2));
       assertFalse(c.namespaceOperations().listIterators(namespace).containsKey(iterName));
       assertFalse(c.tableOperations().listIterators(t1).containsKey(iterName));
     }
@@ -898,8 +896,8 @@
     // nobody should have any of these properties yet
     assertFalse(c.instanceOperations().getSystemConfiguration().containsValue(v));
     assertFalse(checkNamespaceHasProp(Namespace.ACCUMULO.name(), k, v));
-    assertFalse(checkTableHasProp(RootTable.NAME, k, v));
-    assertFalse(checkTableHasProp(MetadataTable.NAME, k, v));
+    assertFalse(checkTableHasProp(AccumuloTable.ROOT.tableName(), k, v));
+    assertFalse(checkTableHasProp(AccumuloTable.METADATA.tableName(), k, v));
     assertFalse(checkNamespaceHasProp(Namespace.DEFAULT.name(), k, v));
     assertFalse(checkTableHasProp(defaultNamespaceTable, k, v));
     assertFalse(checkNamespaceHasProp(namespace, k, v));
@@ -908,12 +906,14 @@
     // set the filter, verify that accumulo namespace is the only one unaffected
     c.instanceOperations().setProperty(k, v);
     // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
-    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
+    Thread.sleep(250);
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(v));
     assertEquals(systemNamespaceShouldInherit,
         checkNamespaceHasProp(Namespace.ACCUMULO.name(), k, v));
-    assertEquals(systemNamespaceShouldInherit, checkTableHasProp(RootTable.NAME, k, v));
-    assertEquals(systemNamespaceShouldInherit, checkTableHasProp(MetadataTable.NAME, k, v));
+    assertEquals(systemNamespaceShouldInherit,
+        checkTableHasProp(AccumuloTable.ROOT.tableName(), k, v));
+    assertEquals(systemNamespaceShouldInherit,
+        checkTableHasProp(AccumuloTable.METADATA.tableName(), k, v));
     assertTrue(checkNamespaceHasProp(Namespace.DEFAULT.name(), k, v));
     assertTrue(checkTableHasProp(defaultNamespaceTable, k, v));
     assertTrue(checkNamespaceHasProp(namespace, k, v));
@@ -922,11 +922,11 @@
     // verify it is no longer inherited
     c.instanceOperations().removeProperty(k);
     // doesn't take effect immediately, needs time to propagate to tserver's ZooKeeper cache
-    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
+    Thread.sleep(250);
     assertFalse(c.instanceOperations().getSystemConfiguration().containsValue(v));
     assertFalse(checkNamespaceHasProp(Namespace.ACCUMULO.name(), k, v));
-    assertFalse(checkTableHasProp(RootTable.NAME, k, v));
-    assertFalse(checkTableHasProp(MetadataTable.NAME, k, v));
+    assertFalse(checkTableHasProp(AccumuloTable.ROOT.tableName(), k, v));
+    assertFalse(checkTableHasProp(AccumuloTable.METADATA.tableName(), k, v));
     assertFalse(checkNamespaceHasProp(Namespace.DEFAULT.name(), k, v));
     assertFalse(checkTableHasProp(defaultNamespaceTable, k, v));
     assertFalse(checkNamespaceHasProp(namespace, k, v));
diff --git a/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java b/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java
index 93375d8..dea8973 100644
--- a/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java
@@ -32,7 +32,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -95,7 +95,7 @@
       }
 
       // ensure that the recovery was not a merging minor compaction
-      try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.fetchColumnFamily(DataFileColumnFamily.NAME);
         for (Entry<Key,Value> entry : s) {
           String filename = entry.getKey().getColumnQualifier().toString();
diff --git a/test/src/main/java/org/apache/accumulo/test/SampleIT.java b/test/src/main/java/org/apache/accumulo/test/SampleIT.java
index 369444a..c4f4a5f 100644
--- a/test/src/main/java/org/apache/accumulo/test/SampleIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SampleIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test;
 
+import static org.apache.accumulo.test.util.FileMetadataUtil.countFiles;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
@@ -26,11 +27,13 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -52,6 +55,7 @@
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.OfflineScanner;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -63,6 +67,8 @@
 import org.apache.accumulo.core.iterators.WrappingIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.test.util.FileMetadataUtil;
+import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Iterables;
@@ -121,13 +127,74 @@
   }
 
   @Test
-  public void testBasic() throws Exception {
+  public void testSampleFencing() throws Exception {
 
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
       String clone = tableName + "_clone";
 
-      client.tableOperations().create(tableName, new NewTableConfiguration().enableSampling(SC1));
+      createTable(client, tableName, new NewTableConfiguration().enableSampling(SC1));
+
+      BatchWriter bw = client.createBatchWriter(tableName);
+
+      TreeMap<Key,Value> expected = new TreeMap<>();
+      writeData(bw, SC1, expected);
+      assertEquals(20, expected.size());
+
+      Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY);
+      Scanner isoScanner =
+          new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY));
+      Scanner csiScanner =
+          new ClientSideIteratorScanner(client.createScanner(tableName, Authorizations.EMPTY));
+      scanner.setSamplerConfiguration(SC1);
+      csiScanner.setSamplerConfiguration(SC1);
+      isoScanner.setSamplerConfiguration(SC1);
+      isoScanner.setBatchSize(10);
+
+      try (BatchScanner bScanner = client.createBatchScanner(tableName)) {
+        bScanner.setSamplerConfiguration(SC1);
+        bScanner.setRanges(Arrays.asList(new Range()));
+
+        check(expected, scanner, bScanner, isoScanner, csiScanner);
+
+        client.tableOperations().flush(tableName, null, null, true);
+
+        // Fence off the data to a Range that is a subset of the original data
+        Range fenced = new Range(new Text(String.format("r_%06d", 3000)),
+            new Text(String.format("r_%06d", 6000)));
+        FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, Set.of(fenced));
+        assertEquals(1, countFiles(getServerContext(), tableName));
+
+        // Build the map of expected values to be seen by filtering out keys not in the fenced range
+        TreeMap<Key,Value> fenceExpected =
+            expected.entrySet().stream().filter(entry -> fenced.contains(entry.getKey())).collect(
+                Collectors.toMap(Entry::getKey, Entry::getValue, (v1, v2) -> v1, TreeMap::new));
+
+        Scanner oScanner = newOfflineScanner(client, tableName, clone, SC1);
+
+        // verify only the correct values in the fenced range are seen
+        check(fenceExpected, scanner, bScanner, isoScanner, csiScanner, oScanner);
+      }
+    }
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    testBasic(Set.of());
+  }
+
+  @Test
+  public void testBasicWithFencedFiles() throws Exception {
+    testBasic(createRanges());
+  }
+
+  private void testBasic(Set<Range> fileRanges) throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      String clone = tableName + "_clone";
+
+      createTable(client, tableName, new NewTableConfiguration().enableSampling(SC1));
 
       BatchWriter bw = client.createBatchWriter(tableName);
 
@@ -153,6 +220,12 @@
 
         client.tableOperations().flush(tableName, null, null, true);
 
+        // Split files into ranged files if provided
+        if (!fileRanges.isEmpty()) {
+          FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, fileRanges);
+          assertEquals(fileRanges.size(), countFiles(getServerContext(), tableName));
+        }
+
         Scanner oScanner = newOfflineScanner(client, tableName, clone, SC1);
         check(expected, scanner, bScanner, isoScanner, csiScanner, oScanner);
 
@@ -293,11 +366,20 @@
 
   @Test
   public void testIterator() throws Exception {
+    testIterator(Set.of());
+  }
+
+  @Test
+  public void testIteratorFencedFiles() throws Exception {
+    testIterator(createRanges());
+  }
+
+  private void testIterator(Set<Range> fileRanges) throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
       String clone = tableName + "_clone";
 
-      client.tableOperations().create(tableName, new NewTableConfiguration().enableSampling(SC1));
+      createTable(client, tableName, new NewTableConfiguration().enableSampling(SC1));
 
       TreeMap<Key,Value> expected = new TreeMap<>();
       try (BatchWriter bw = client.createBatchWriter(tableName)) {
@@ -346,6 +428,12 @@
         // flush an rerun same test against files
         client.tableOperations().flush(tableName, null, null, true);
 
+        // Split files into ranged files if provided
+        if (!fileRanges.isEmpty()) {
+          FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, fileRanges);
+          assertEquals(fileRanges.size(), countFiles(getServerContext(), tableName));
+        }
+
         oScanner = newOfflineScanner(client, tableName, clone, null);
         oScanner.addScanIterator(new IteratorSetting(100, IteratorThatUsesSample.class));
         scanners = Arrays.asList(scanner, isoScanner, bScanner, csiScanner, oScanner);
@@ -401,12 +489,21 @@
 
   @Test
   public void testSampleNotPresent() throws Exception {
+    testSampleNotPresent(Set.of());
+  }
+
+  @Test
+  public void testSampleNotPresentFencedFiles() throws Exception {
+    testSampleNotPresent(createRanges());
+  }
+
+  private void testSampleNotPresent(Set<Range> fileRanges) throws Exception {
 
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
       String clone = tableName + "_clone";
 
-      client.tableOperations().create(tableName);
+      createTable(client, tableName, new NewTableConfiguration());
 
       TreeMap<Key,Value> expected = new TreeMap<>();
       try (BatchWriter bw = client.createBatchWriter(tableName)) {
@@ -425,12 +522,17 @@
 
         client.tableOperations().flush(tableName, null, null, true);
 
+        // Split files into ranged files if provided
+        if (!fileRanges.isEmpty()) {
+          FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, fileRanges);
+          assertEquals(fileRanges.size(), countFiles(getServerContext(), tableName));
+        }
+
         Scanner oScanner = newOfflineScanner(client, tableName, clone, SC1);
         assertSampleNotPresent(SC1, scanner, isoScanner, bScanner, csiScanner, oScanner);
 
         // configure sampling, however there exist an rfile w/o sample data... so should still see
         // sample not present exception
-
         updateSamplingConfig(client, tableName, SC1);
 
         // create clone with new config
@@ -509,4 +611,30 @@
           actual.size(), expected.size(), s.getClass().getSimpleName()));
     }
   }
+
+  private Set<Range> createRanges() {
+    Set<Range> ranges = new HashSet<>();
+
+    int splits = 10;
+
+    for (int i = 0; i < splits; i++) {
+      Text start = i > 0 ? new Text(String.format("r_%06d", i * 1000)) : null;
+      Text end = i < splits - 1 ? new Text(String.format("r_%06d", (i + 1) * 1000)) : null;
+      ranges.add(new Range(start, end));
+    }
+
+    return ranges;
+  }
+
+  // Create a table and disable compactions. This is important to prevent intermittent
+  // failures when testing if sampling is configured or not. Some of the tests first
+  // assert sampling is not available, then configures sampling and tests it still isn't
+  // available before triggering a compaction to confirm it is now available. Intermittent
+  // GCs can make these tests non-deterministic when there are a lot of files created
+  // during the fencing tests.
+  private void createTable(AccumuloClient client, String tableName, NewTableConfiguration ntc)
+      throws Exception {
+    ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "9999"));
+    client.tableOperations().create(tableName, ntc);
+  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java
index 0c03de9..85d0f48 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java
@@ -438,16 +438,13 @@
       this.tctx = testContext;
     }
 
-    @SuppressWarnings("deprecation")
     private long bulkImport(Random random, Collection<Mutation> mutations) throws Exception {
 
       if (mutations.isEmpty()) {
         return 0;
       }
 
-      String name = "/bulkimport_" + nextLongAbs(random);
-      Path bulkDir = new Path(tctx.tmpDir + name);
-      Path failDir = new Path(tctx.tmpDir + name + "_failures");
+      Path bulkDir = new Path(tctx.tmpDir + "/bulkimport_" + nextLongAbs(random));
 
       List<Key> keys = mutations.stream().flatMap(ScanConsistencyIT::toKeys).sorted()
           .collect(Collectors.toList());
@@ -463,22 +460,10 @@
           }
         }
 
-        if (random.nextBoolean()) {
-          // use bulk import v1
-          tctx.fileSystem.mkdirs(failDir);
-          tctx.client.tableOperations().importDirectory(tctx.table, bulkDir.toString(),
-              failDir.toString(), true);
-          assertEquals(0, tctx.fileSystem.listStatus(failDir).length,
-              "Failure dir was not empty " + failDir);
-        } else {
-          // use bulk import v2
-          tctx.client.tableOperations().importDirectory(bulkDir.toString()).to(tctx.table)
-              .tableTime(true).load();
-        }
-
+        tctx.client.tableOperations().importDirectory(bulkDir.toString()).to(tctx.table)
+            .tableTime(true).load();
       } finally {
         tctx.fileSystem.delete(bulkDir, true);
-        tctx.fileSystem.delete(failDir, true);
       }
 
       return keys.size();
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
index 7c88ee4..fdbaf73 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanFlushWithTimeIT.java
@@ -37,7 +37,6 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.io.Text;
@@ -69,7 +68,7 @@
       partitionKeys.add(new Text("5"));
       c.tableOperations().addSplits(tableName, partitionKeys);
       log.info("waiting for zookeeper propagation");
-      UtilWaitThread.sleep(5_000);
+      Thread.sleep(5_000);
       log.info("Adding a few entries");
       try (BatchWriter bw = c.createBatchWriter(tableName)) {
         for (int i = 0; i < 10; i++) {
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java
index 6cd6ba7..a11385d 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java
@@ -28,7 +28,6 @@
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.util.ScanServerMetadataEntries;
@@ -37,6 +36,8 @@
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 @Tag(MINI_CLUSTER_ONLY)
 public class ScanServerMetadataEntriesCleanIT extends SharedMiniClusterBase {
 
diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java
index 7fbe060..ab92e57 100644
--- a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java
@@ -48,11 +48,11 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.gc.Reference;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.gc.GCRun;
 import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
@@ -66,6 +66,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 @Tag(MINI_CLUSTER_ONLY)
 public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
 
@@ -232,7 +234,8 @@
         assertNotNull(iter.next());
 
         List<Entry<Key,Value>> metadataEntries = null;
-        try (Scanner scanner2 = client.createScanner("accumulo.metadata", Authorizations.EMPTY)) {
+        try (Scanner scanner2 =
+            client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
           scanner2.setRange(ScanServerFileReferenceSection.getRange());
           metadataEntries = scanner2.stream().distinct().collect(Collectors.toList());
         }
@@ -249,12 +252,14 @@
         assertEquals(fileCount, metadataScanFileRefs.size());
 
         assertEquals(fileCount, ctx.getAmple().getScanServerFileReferences().count());
-
-        List<Reference> refs = gc.getReferences().collect(Collectors.toList());
+        List<Reference> refs;
+        try (Stream<Reference> references = gc.getReferences()) {
+          refs = references.collect(Collectors.toList());
+        }
         assertTrue(refs.size() > fileCount * 2);
         List<Reference> tableRefs =
             refs.stream().filter(r -> r.getTableId().equals(tid) && !r.isDirectory())
-                .peek(r -> assertTrue(metadataScanFileRefs.contains(r.getMetadataEntry())))
+                .peek(r -> assertTrue(metadataScanFileRefs.contains(r.getMetadataPath())))
                 .collect(Collectors.toList());
         log.info("Reference List:{}", tableRefs);
         // There should be 6 references here. 3 for the table file entries, and 3 for the scan
@@ -262,7 +267,7 @@
         assertEquals(fileCount * 2, tableRefs.size());
 
         Set<String> deduplicatedReferences =
-            tableRefs.stream().map(Reference::getMetadataEntry).collect(Collectors.toSet());
+            tableRefs.stream().map(Reference::getMetadataPath).collect(Collectors.toSet());
 
         assertEquals(fileCount, deduplicatedReferences.size());
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
index 926f0b9..265aa79 100644
--- a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.test;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
@@ -36,7 +34,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
@@ -62,7 +60,8 @@
 
   boolean isOffline(String tablename, AccumuloClient client) throws TableNotFoundException {
     String tableId = client.tableOperations().tableIdMap().get(tablename);
-    try (Scanner scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       scanner.setRange(new Range(new Text(tableId + ";"), new Text(tableId + "<")));
       scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
       return scanner.stream().findAny().isEmpty();
@@ -86,12 +85,12 @@
         // take the table offline
         client.tableOperations().offline(tableName);
         while (!isOffline(tableName, client)) {
-          sleepUninterruptibly(200, MILLISECONDS);
+          Thread.sleep(200);
         }
 
         // poke a partial split into the metadata table
-        client.securityOperations().grantTablePermission(getAdminPrincipal(), MetadataTable.NAME,
-            TablePermission.WRITE);
+        client.securityOperations().grantTablePermission(getAdminPrincipal(),
+            AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
         TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
 
         KeyExtent extent = new KeyExtent(tableId, null, new Text("b"));
@@ -99,13 +98,13 @@
 
         TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(0.5)));
         TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, TabletColumnFamily.encodePrevEndRow(null));
-        try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
+        try (BatchWriter bw = client.createBatchWriter(AccumuloTable.METADATA.tableName())) {
           bw.addMutation(m);
 
           if (tn == 1) {
             bw.flush();
 
-            try (Scanner scanner = client.createScanner(MetadataTable.NAME)) {
+            try (Scanner scanner = client.createScanner(AccumuloTable.METADATA.tableName())) {
               scanner.setRange(extent.toMetaRange());
               scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java b/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java
index 95738e8..90f81ee 100644
--- a/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TableConfigurationUpdateIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
@@ -124,7 +125,7 @@
 
       String t = Thread.currentThread().getName() + " ";
       try {
-        random.ints(iterations, 0, randMax).forEach(choice -> {
+        RANDOM.get().ints(iterations, 0, randMax).forEach(choice -> {
           if (choice < 1) {
             tableConf.invalidateCache();
           } else {
diff --git a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java b/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
index 82bb567..df8151b 100644
--- a/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TableOperationsIT.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -39,7 +39,6 @@
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -60,6 +59,7 @@
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.constraints.DefaultKeySizeConstraint;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -286,15 +286,17 @@
   /** Test recovery from bad majc iterator via compaction cancel. */
   @Test
   public void testCompactEmptyTablesWithBadIterator_FailsAndCancel() throws TableExistsException,
-      AccumuloException, AccumuloSecurityException, TableNotFoundException {
+      AccumuloException, AccumuloSecurityException, TableNotFoundException, InterruptedException {
     String tableName = getUniqueNames(1)[0];
     accumuloClient.tableOperations().create(tableName);
 
     List<IteratorSetting> list = new ArrayList<>();
     list.add(new IteratorSetting(15, BadIterator.class));
-    accumuloClient.tableOperations().compact(tableName, null, null, list, true, false); // don't
-                                                                                        // block
-    sleepUninterruptibly(2, TimeUnit.SECONDS); // start compaction
+    // don't block
+    accumuloClient.tableOperations().compact(tableName, null, null, list, true, false);
+
+    Thread.sleep(SECONDS.toMillis(2)); // start compaction
+
     accumuloClient.tableOperations().cancelCompaction(tableName);
 
     try (Scanner scanner = accumuloClient.createScanner(tableName, Authorizations.EMPTY)) {
@@ -353,13 +355,10 @@
     assertEquals(TimeType.LOGICAL, timeType);
 
     // check system tables
-    timeType = accumuloClient.tableOperations().getTimeType("accumulo.metadata");
+    timeType = accumuloClient.tableOperations().getTimeType(AccumuloTable.METADATA.tableName());
     assertEquals(TimeType.LOGICAL, timeType);
 
-    timeType = accumuloClient.tableOperations().getTimeType("accumulo.replication");
-    assertEquals(TimeType.LOGICAL, timeType);
-
-    timeType = accumuloClient.tableOperations().getTimeType("accumulo.root");
+    timeType = accumuloClient.tableOperations().getTimeType(AccumuloTable.ROOT.tableName());
     assertEquals(TimeType.LOGICAL, timeType);
 
     // test non-existent table
diff --git a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java b/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
index 0f37f34..a6bb86e 100644
--- a/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TabletServerGivesUpIT.java
@@ -18,11 +18,10 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.time.Duration;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -77,7 +76,7 @@
       splitter.start();
       // wait for the tserver to give up on writing to the WAL
       while (client.instanceOperations().getTabletServers().size() == 1) {
-        sleepUninterruptibly(1, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(1));
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
index b074ce2..7fea809 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.test;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.harness.AccumuloITBase.random;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.util.Iterator;
 import java.util.Map.Entry;
@@ -166,7 +166,7 @@
       long t1 = System.currentTimeMillis();
 
       for (int i = 0; i < numLookups; i++) {
-        long row = ((random.nextLong() & 0x7fffffffffffffffL) % opts.num) + opts.start;
+        long row = ((RANDOM.get().nextLong() & 0x7fffffffffffffffL) % opts.num) + opts.start;
 
         try (Scanner s = accumuloClient.createScanner(opts.tableName, opts.auths)) {
           Key startKey = new Key(encodeLong(row), CF_BYTES, CQ_BYTES, new byte[0], Long.MAX_VALUE);
diff --git a/test/src/main/java/org/apache/accumulo/test/TestIngest.java b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
index f20cf31..7f44ad0 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestIngest.java
@@ -49,11 +49,13 @@
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.rfile.RFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -275,8 +277,9 @@
     if (params.outputFile != null) {
       ClientContext cc = (ClientContext) accumuloClient;
       writer = FileOperations.getInstance().newWriterBuilder()
-          .forFile(params.outputFile + "." + RFile.EXTENSION, fs, cc.getHadoopConf(),
-              NoCryptoServiceFactory.NONE)
+          .forFile(
+              UnreferencedTabletFile.of(fs, new Path(params.outputFile + "." + RFile.EXTENSION)),
+              fs, cc.getHadoopConf(), NoCryptoServiceFactory.NONE)
           .withTableConfiguration(DefaultConfiguration.getInstance()).build();
       writer.startDefaultLocalityGroup();
     } else {
diff --git a/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java b/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
index 3caab70..d852963 100644
--- a/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
+++ b/test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.test;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
@@ -31,7 +33,7 @@
 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.util.UtilWaitThread;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.io.Text;
 
 import com.beust.jcommander.Parameter;
@@ -54,9 +56,7 @@
     int i = 0;
     for (String table : tableNames) {
       // wait for table to exist
-      while (!client.tableOperations().exists(table)) {
-        UtilWaitThread.sleep(100);
-      }
+      Wait.waitFor(() -> client.tableOperations().exists(table), SECONDS.toMillis(30), 100);
       try (Scanner scanner = client.createScanner(table, opts.auths)) {
         int count = i;
         for (Entry<Key,Value> elt : scanner) {
diff --git a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
index 91c7884..65617b2 100644
--- a/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TotalQueuedIT.java
@@ -18,11 +18,11 @@
  */
 package org.apache.accumulo.test;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -30,17 +30,18 @@
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class TotalQueuedIT extends ConfigurableMacBase {
 
   @Override
@@ -67,19 +68,19 @@
       c.tableOperations().create(tableName);
       c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "9999");
       c.tableOperations().setProperty(tableName, Property.TABLE_FILE_MAX.getKey(), "999");
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
       // get an idea of how fast the syncs occur
       byte[] row = new byte[250];
       BatchWriterConfig cfg = new BatchWriterConfig();
       cfg.setMaxWriteThreads(10);
-      cfg.setMaxLatency(1, TimeUnit.SECONDS);
+      cfg.setMaxLatency(1, SECONDS);
       cfg.setMaxMemory(1024 * 1024);
       long realSyncs = getSyncs(c);
       long now = System.currentTimeMillis();
       long bytesSent = 0;
       try (BatchWriter bw = c.createBatchWriter(tableName, cfg)) {
         for (int i = 0; i < N; i++) {
-          random.nextBytes(row);
+          RANDOM.get().nextBytes(row);
           Mutation m = new Mutation(row);
           m.put("", "", "");
           bw.addMutation(m);
@@ -100,12 +101,12 @@
       c.instanceOperations().setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX.getKey(),
           "" + LARGE_QUEUE_SIZE);
       c.tableOperations().flush(tableName, null, null, true);
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
       try (BatchWriter bw = c.createBatchWriter(tableName, cfg)) {
         now = System.currentTimeMillis();
         bytesSent = 0;
         for (int i = 0; i < N; i++) {
-          random.nextBytes(row);
+          RANDOM.get().nextBytes(row);
           Mutation m = new Mutation(row);
           m.put("", "", "");
           bw.addMutation(m);
@@ -127,8 +128,8 @@
   private long getSyncs(AccumuloClient c) throws Exception {
     ServerContext context = getServerContext();
     for (String address : c.instanceOperations().getTabletServers()) {
-      TabletClientService.Client client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER,
-          HostAndPort.fromString(address), context);
+      TabletServerClientService.Client client = ThriftUtil
+          .getClient(ThriftClientTypes.TABLET_SERVER, HostAndPort.fromString(address), context);
       TabletServerStatus status = client.getTabletServerStatus(null, context.rpcCreds());
       return status.syncs;
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java b/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java
index 9c3474f..5396db6 100644
--- a/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/TransportCachingIT.java
@@ -33,7 +33,6 @@
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
 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.harness.AccumuloClusterHarness;
 import org.apache.thrift.transport.TTransport;
@@ -42,6 +41,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Test that {@link ThriftTransportPool} actually adheres to the cachedConnection argument
  */
diff --git a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
index 4e99e59..e982f90 100644
--- a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -35,7 +35,7 @@
 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.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
@@ -104,7 +104,8 @@
       getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
 
       // wait for the metadata table to be online
-      try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.forEach((k, v) -> {});
       }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
index 3dc8b27..49148b8 100644
--- a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -34,6 +35,7 @@
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl.ProcessInfo;
@@ -55,7 +57,7 @@
   public static byte[] randomHex(int n) {
     byte[] binary = new byte[n];
     byte[] hex = new byte[n * 2];
-    random.nextBytes(binary);
+    RANDOM.get().nextBytes(binary);
     int count = 0;
     for (byte x : binary) {
       hex[count++] = HEXCHARS[(x >> 4) & 0xf];
@@ -125,7 +127,8 @@
           Pattern.compile(".*recovered \\d+ mutations creating \\d+ entries from \\d+ walogs.*");
       for (String line : result.split("\n")) {
         // ignore metadata tables
-        if (line.contains("!0") || line.contains("+r")) {
+        if (line.contains(AccumuloTable.METADATA.tableId().canonical())
+            || line.contains(AccumuloTable.ROOT.tableId().canonical())) {
           continue;
         }
         if (line.contains("recovering data from walogs")) {
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
index a0cba27..531de27 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -45,7 +45,7 @@
 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.metadata.MetadataTable;
+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.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
@@ -184,7 +184,8 @@
 
     TreeSet<String> volumesSeen = new TreeSet<>();
     int fileCount = 0;
-    try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        accumuloClient.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       scanner.setRange(tableRange);
       scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
       for (Entry<Key,Value> entry : scanner) {
@@ -207,7 +208,8 @@
 
   public static void verifyNoVolumes(AccumuloClient accumuloClient, Range tableRange)
       throws Exception {
-    try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        accumuloClient.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       scanner.setRange(tableRange);
       scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
       for (Entry<Key,Value> entry : scanner) {
@@ -248,7 +250,8 @@
     Collections.addAll(volumes, vol.split(","));
 
     TreeSet<String> volumesSeen = new TreeSet<>();
-    try (Scanner scanner = accumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        accumuloClient.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       scanner.setRange(tableRange);
       scanner.fetchColumnFamily(LogColumnFamily.NAME);
       for (Entry<Key,Value> entry : scanner) {
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
index 3c4aa4b..79c6f6d 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
@@ -36,6 +36,7 @@
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 
 import org.apache.accumulo.core.Constants;
@@ -61,18 +62,23 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.init.Initialize;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
 import org.apache.accumulo.server.log.WalStateManager.WalState;
+import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.accumulo.test.util.FileMetadataUtil;
 import org.apache.commons.configuration2.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -142,14 +148,17 @@
         }
       }
       // verify the new files are written to the different volumes
-      try (Scanner scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.setRange(new Range("1", "1<"));
         scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
         int fileCount = 0;
 
         for (Entry<Key,Value> entry : scanner) {
-          boolean inV1 = entry.getKey().getColumnQualifier().toString().contains(v1.toString());
-          boolean inV2 = entry.getKey().getColumnQualifier().toString().contains(v2.toString());
+          boolean inV1 = StoredTabletFile.of(entry.getKey().getColumnQualifier()).getMetadataPath()
+              .contains(v1.toString());
+          boolean inV2 = StoredTabletFile.of(entry.getKey().getColumnQualifier()).getMetadataPath()
+              .contains(v2.toString());
           assertTrue(inV1 || inV2);
           fileCount++;
         }
@@ -276,6 +285,11 @@
 
   private void verifyVolumesUsed(AccumuloClient client, String tableName, boolean shouldExist,
       Path... paths) throws Exception {
+    verifyVolumesUsed(client, tableName, shouldExist, false, paths);
+  }
+
+  private void verifyVolumesUsed(AccumuloClient client, String tableName, boolean shouldExist,
+      boolean rangedFiles, Path... paths) throws Exception {
 
     if (!client.tableOperations().exists(tableName)) {
       assertFalse(shouldExist);
@@ -290,17 +304,18 @@
     verifyData(expected, client.createScanner(tableName, Authorizations.EMPTY));
 
     TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
-    try (Scanner metaScanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner metaScanner =
+        client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       metaScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
       metaScanner.setRange(new KeyExtent(tableId, null, null).toMetaRange());
 
       int[] counts = new int[paths.length];
 
       outer: for (Entry<Key,Value> entry : metaScanner) {
-        String path = entry.getKey().getColumnQualifier().toString();
+        String path = StoredTabletFile.of(entry.getKey().getColumnQualifier()).getMetadataPath();
 
         for (int i = 0; i < paths.length; i++) {
-          if (path.startsWith(paths[i].toString())) {
+          if (path.contains(paths[i].toString())) {
             counts[i]++;
             continue outer;
           }
@@ -344,7 +359,10 @@
         sum += count;
       }
 
-      assertEquals(100, sum);
+      // When ranged files exist we there should be twice as many
+      // as the test split each file into 2
+      int expectedCount = rangedFiles ? 200 : 100;
+      assertEquals(expectedCount, sum);
     }
   }
 
@@ -367,13 +385,14 @@
 
       verifyVolumesUsed(client, tableNames[0], true, v2);
 
-      client.tableOperations().compact(RootTable.NAME, new CompactionConfig().setWait(true));
+      client.tableOperations().compact(AccumuloTable.ROOT.tableName(),
+          new CompactionConfig().setWait(true));
 
       // check that root tablet is not on volume 1
       int count = 0;
       for (StoredTabletFile file : ((ClientContext) client).getAmple().readTablet(RootTable.EXTENT)
           .getFiles()) {
-        assertTrue(file.getMetaUpdateDelete().startsWith(v2.toString()));
+        assertTrue(file.getMetadataPath().startsWith(v2.toString()));
         count++;
       }
 
@@ -382,15 +401,16 @@
       client.tableOperations().clone(tableNames[0], tableNames[1], true, new HashMap<>(),
           new HashSet<>());
 
-      client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-      client.tableOperations().flush(RootTable.NAME, null, null, true);
+      client.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+      client.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
 
       verifyVolumesUsed(client, tableNames[0], true, v2);
       verifyVolumesUsed(client, tableNames[1], true, v2);
     }
   }
 
-  private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown) throws Exception {
+  private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown, boolean rangedFiles)
+      throws Exception {
     String[] tableNames = getUniqueNames(3);
 
     verifyVolumesUsed(client, tableNames[0], false, v1, v2);
@@ -401,6 +421,13 @@
       writeData(tableNames[1], c2);
     }
 
+    // If flag is true then for each file split and create two files
+    // to verify volume replacement works on files with ranges
+    if (rangedFiles) {
+      splitFilesWithRange(client, tableNames[0]);
+      splitFilesWithRange(client, tableNames[1]);
+    }
+
     if (cleanShutdown) {
       assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     }
@@ -426,24 +453,26 @@
     // start cluster and verify that volumes were replaced
     cluster.start();
 
-    verifyVolumesUsed(client, tableNames[0], true, v8, v9);
-    verifyVolumesUsed(client, tableNames[1], true, v8, v9);
+    verifyVolumesUsed(client, tableNames[0], true, rangedFiles, v8, v9);
+    verifyVolumesUsed(client, tableNames[1], true, rangedFiles, v8, v9);
 
     // verify writes to new dir
     client.tableOperations().compact(tableNames[0], null, null, true, true);
     client.tableOperations().compact(tableNames[1], null, null, true, true);
 
-    verifyVolumesUsed(client, tableNames[0], true, v8, v9);
-    verifyVolumesUsed(client, tableNames[1], true, v8, v9);
+    // Always pass false for ranged files as compaction will clean them up if exist
+    verifyVolumesUsed(client, tableNames[0], true, false, v8, v9);
+    verifyVolumesUsed(client, tableNames[1], true, false, v8, v9);
 
-    client.tableOperations().compact(RootTable.NAME, new CompactionConfig().setWait(true));
+    client.tableOperations().compact(AccumuloTable.ROOT.tableName(),
+        new CompactionConfig().setWait(true));
 
     // check that root tablet is not on volume 1 or 2
     int count = 0;
     for (StoredTabletFile file : ((ClientContext) client).getAmple().readTablet(RootTable.EXTENT)
         .getFiles()) {
-      assertTrue(file.getMetaUpdateDelete().startsWith(v8.toString())
-          || file.getMetaUpdateDelete().startsWith(v9.toString()));
+      assertTrue(file.getMetadataPath().startsWith(v8.toString())
+          || file.getMetadataPath().startsWith(v9.toString()));
       count++;
     }
 
@@ -452,8 +481,8 @@
     client.tableOperations().clone(tableNames[1], tableNames[2], true, new HashMap<>(),
         new HashSet<>());
 
-    client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-    client.tableOperations().flush(RootTable.NAME, null, null, true);
+    client.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+    client.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
 
     verifyVolumesUsed(client, tableNames[0], true, v8, v9);
     verifyVolumesUsed(client, tableNames[1], true, v8, v9);
@@ -463,14 +492,28 @@
   @Test
   public void testCleanReplaceVolumes() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      testReplaceVolume(client, true);
+      testReplaceVolume(client, true, false);
     }
   }
 
   @Test
   public void testDirtyReplaceVolumes() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      testReplaceVolume(client, false);
+      testReplaceVolume(client, false, false);
+    }
+  }
+
+  @Test
+  public void testCleanReplaceVolumesWithRangedFiles() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
+      testReplaceVolume(client, true, true);
+    }
+  }
+
+  @Test
+  public void testDirtyReplaceVolumesWithRangedFiles() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
+      testReplaceVolume(client, false, true);
     }
   }
 
@@ -486,4 +529,44 @@
       config.write(out);
     }
   }
+
+  // Go through each tablet file in metadata and split the files into two files
+  // by adding two new entries that covers half of the file. This will test that
+  // files with ranges work properly with volume replacement
+  private void splitFilesWithRange(AccumuloClient client, String tableName) throws Exception {
+    client.securityOperations().grantTablePermission(cluster.getConfig().getRootUserName(),
+        AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+    final ServerContext ctx = getServerContext();
+    ctx.setCredentials(new SystemCredentials(client.instanceOperations().getInstanceId(), "root",
+        new PasswordToken(ROOT_PASSWORD)));
+
+    AtomicInteger i = new AtomicInteger();
+    FileMetadataUtil.mutateTabletFiles(ctx, tableName, null, null, (tm, mutator, file, value) -> {
+      i.incrementAndGet();
+
+      // Create a mutation to delete the existing file metadata entry with infinite range
+      mutator.deleteFile(file);
+
+      // Find the midpoint and create two new files, each with a range covering half the file
+      Text tabletMidPoint = getTabletMidPoint(tm.getExtent().endRow());
+      // Handle edge case for last tablet
+      if (tabletMidPoint == null) {
+        tabletMidPoint = new Text(
+            String.format("%06d", Integer.parseInt(tm.getExtent().prevEndRow().toString()) + 50));
+      }
+
+      final DataFileValue newValue = new DataFileValue(Integer.max(1, (int) (value.getSize() / 2)),
+          Integer.max(1, (int) (value.getNumEntries() / 2)));
+      mutator.putFile(StoredTabletFile.of(file.getPath(),
+          new Range(tm.getExtent().prevEndRow(), tabletMidPoint)), newValue);
+      mutator.putFile(
+          StoredTabletFile.of(file.getPath(), new Range(tabletMidPoint, tm.getExtent().endRow())),
+          newValue);
+    });
+  }
+
+  private static Text getTabletMidPoint(Text row) {
+    return row != null ? new Text(String.format("%06d", Integer.parseInt(row.toString()) - 50))
+        : null;
+  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java
index 6961453..95a7772 100644
--- a/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java
@@ -33,8 +33,7 @@
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-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.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -56,7 +55,8 @@
   public void test() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
       // ensure the metadata table is online
-      try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.forEach((k, v) -> {});
       }
       c.instanceOperations().waitForBalance();
@@ -78,7 +78,8 @@
   private boolean isBalanced(AccumuloClient c) throws Exception {
     final Map<String,Integer> counts = new HashMap<>();
     int offline = 0;
-    for (String tableName : new String[] {MetadataTable.NAME, RootTable.NAME}) {
+    for (String tableName : new String[] {AccumuloTable.METADATA.tableName(),
+        AccumuloTable.ROOT.tableName()}) {
       try (Scanner s = c.createScanner(tableName, Authorizations.EMPTY)) {
         s.setRange(TabletsSection.getRange());
         s.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
@@ -104,8 +105,7 @@
         }
       }
     }
-    // the replication table is expected to be offline for this test, so ignore it
-    if (offline > 1) {
+    if (offline > 0) {
       System.out.println("Offline tablets " + offline);
       return false;
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java b/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java
index 7c5324c..34aa1de 100644
--- a/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java
@@ -45,7 +45,6 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.constraints.Constraint;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -86,7 +85,11 @@
 
       // the purpose of this constraint is to just randomly hold up inserts on the server side
       if (rand.nextBoolean()) {
-        UtilWaitThread.sleep(4000);
+        try {
+          Thread.sleep(4000);
+        } catch (InterruptedException ex) {
+          throw new IllegalStateException("Interrupted during sleep", ex);
+        }
       }
 
       return null;
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/BadCompactionServiceConfigIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/BadCompactionServiceConfigIT.java
index ad75c74..35813d9 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/BadCompactionServiceConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/BadCompactionServiceConfigIT.java
@@ -55,7 +55,7 @@
 
 public class BadCompactionServiceConfigIT extends AccumuloClusterHarness {
 
-  private static final String CSP = Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey();
+  private static final String CSP = Property.COMPACTION_SERVICE_PREFIX.getKey();
 
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java
index 60ede3d..6b8dfe2 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java
@@ -57,10 +57,10 @@
       final String table = getUniqueNames(1)[0];
 
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner",
           DefaultCompactionPlanner.class.getName());
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.executors",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.executors",
           ("[{'name':'small','type':'internal','maxSize':'2M','numThreads':2},"
               + "{'name':'medium','type':'internal','maxSize':'128M','numThreads':2},"
               + "{'name':'large','type':'internal','numThreads':2}]").replaceAll("'", "\""));
@@ -91,7 +91,7 @@
       // compactions. Because the compactions are running slow, expect this config change to overlap
       // with running compactions.
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.executors",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.executors",
           ("[{'name':'little','type':'internal','maxSize':'128M','numThreads':8},"
               + "{'name':'big','type':'internal','numThreads':2}]").replaceAll("'", "\""));
 
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java
index 4697f20..ec9b22f 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.compaction;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -52,6 +53,7 @@
 import org.apache.accumulo.core.client.admin.compaction.TooManyDeletesSelector;
 import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
 import org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer;
+import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -107,7 +109,7 @@
       if (Boolean.parseBoolean(params.getExecutionHints().getOrDefault("compact_all", "false"))) {
         return params
             .createPlanBuilder().addJob((short) 1,
-                executorIds.get(random.nextInt(executorIds.size())), params.getCandidates())
+                executorIds.get(RANDOM.get().nextInt(executorIds.size())), params.getCandidates())
             .build();
       }
 
@@ -120,7 +122,8 @@
         params.getCandidates().stream().collect(Collectors.groupingBy(TestPlanner::getFirstChar))
             .values().forEach(files -> {
               for (int i = filesPerCompaction; i <= files.size(); i += filesPerCompaction) {
-                planBuilder.addJob((short) 1, executorIds.get(random.nextInt(executorIds.size())),
+                planBuilder.addJob((short) 1,
+                    executorIds.get(RANDOM.get().nextInt(executorIds.size())),
                     files.subList(i - filesPerCompaction, i));
               }
             });
@@ -137,7 +140,7 @@
     SharedMiniClusterBase.startMiniClusterWithConfig((miniCfg, coreSite) -> {
       Map<String,String> siteCfg = new HashMap<>();
 
-      var csp = Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey();
+      var csp = Property.COMPACTION_SERVICE_PREFIX.getKey();
       siteCfg.put(csp + "cs1.planner", TestPlanner.class.getName());
       siteCfg.put(csp + "cs1.planner.opts.executors", "3");
       siteCfg.put(csp + "cs1.planner.opts.filesPerCompaction", "5");
@@ -177,7 +180,9 @@
   public void cleanup() {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       client.tableOperations().list().stream()
-          .filter(tableName -> !tableName.startsWith("accumulo.")).forEach(tableName -> {
+          .filter(
+              tableName -> !tableName.startsWith(Namespace.ACCUMULO.name() + Namespace.SEPARATOR))
+          .forEach(tableName -> {
             try {
               client.tableOperations().delete(tableName);
             } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
@@ -200,10 +205,11 @@
 
       assertEquals(2, getFiles(client, "rctt").size());
 
-      client.instanceOperations().setProperty(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey()
-          + "recfg.planner.opts.filesPerCompaction", "5");
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "recfg.planner.opts.executors", "1");
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "recfg.planner.opts.filesPerCompaction",
+          "5");
+      client.instanceOperations().setProperty(
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "recfg.planner.opts.executors", "1");
 
       addFiles(client, "rctt", 10);
 
@@ -218,15 +224,15 @@
   @Test
   public void testAddCompactionService() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      client.instanceOperations().setProperty(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey()
-          + "newcs.planner.opts.filesPerCompaction", "7");
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.process",
-          "SYSTEM");
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.filesPerCompaction",
+          "7");
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.executors", "3");
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.process", "SYSTEM");
       client.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.executors", "3");
+      client.instanceOperations().setProperty(
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner",
           TestPlanner.class.getName());
 
       createTable(client, "acst", "newcs");
@@ -317,6 +323,7 @@
   }
 
   @Test
+  @SuppressWarnings("deprecation")
   public void testTooManyDeletes() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       Map<String,
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingDeprecatedIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingDeprecatedIT.java
deleted file mode 100644
index 56357f8..0000000
--- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingDeprecatedIT.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.test.compaction;
-
-import org.apache.accumulo.core.conf.Property;
-
-public class CompactionRateLimitingDeprecatedIT extends CompactionRateLimitingIT {
-  @Override
-  @SuppressWarnings("removal")
-  protected Property getThroughputProp() {
-    return Property.TSERV_MAJC_THROUGHPUT;
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java
index 630168e..a8dcb54 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.test.compaction;
 
+import static org.apache.accumulo.core.conf.Property.COMPACTION_SERVICE_PREFIX;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.Map;
@@ -38,6 +40,7 @@
   public static final long BYTES_TO_WRITE = 10 * 1024 * 1024;
   public static final long RATE = 1 * 1024 * 1024;
 
+  @SuppressWarnings("deprecation")
   protected Property getThroughputProp() {
     return Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT;
   }
@@ -48,10 +51,10 @@
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "20");
     cfg.setProperty(Property.TABLE_FILE_COMPRESSION_TYPE, "none");
 
-    cfg.setProperty("tserver.compaction.major.service.test.rate.limit", RATE + "B");
-    cfg.setProperty("tserver.compaction.major.service.test.planner",
+    cfg.setProperty(COMPACTION_SERVICE_PREFIX.getKey() + "test.rate.limit", RATE + "B");
+    cfg.setProperty(COMPACTION_SERVICE_PREFIX.getKey() + "test.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.test.planner.opts.executors",
+    cfg.setProperty(COMPACTION_SERVICE_PREFIX.getKey() + "test.planner.opts.executors",
         "[{'name':'all','numThreads':2}]".replaceAll("'", "\""));
 
   }
@@ -76,13 +79,13 @@
         try (BatchWriter bw = client.createBatchWriter(tableName)) {
           while (bytesWritten < BYTES_TO_WRITE) {
             byte[] rowKey = new byte[32];
-            random.nextBytes(rowKey);
+            RANDOM.get().nextBytes(rowKey);
 
             byte[] qual = new byte[32];
-            random.nextBytes(qual);
+            RANDOM.get().nextBytes(qual);
 
             byte[] value = new byte[1024];
-            random.nextBytes(value);
+            RANDOM.get().nextBytes(value);
 
             Mutation m = new Mutation(rowKey);
             m.put(new byte[0], qual, value);
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ConfigurableMajorCompactionIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ConfigurableMajorCompactionIT.java
deleted file mode 100644
index d3c7cf9..0000000
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ConfigurableMajorCompactionIT.java
+++ /dev/null
@@ -1,126 +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.test.compaction;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.CompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-import org.apache.accumulo.tserver.compaction.WriteParameters;
-import org.apache.hadoop.conf.Configuration;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Iterators;
-
-@SuppressWarnings("removal")
-public class ConfigurableMajorCompactionIT extends ConfigurableMacBase {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofSeconds(30);
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    Map<String,String> siteConfig = new HashMap<>();
-    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "1s");
-    cfg.setSiteConfig(siteConfig);
-  }
-
-  public static class TestCompactionStrategy extends CompactionStrategy {
-
-    @Override
-    public boolean shouldCompact(MajorCompactionRequest request) {
-      return request.getFiles().size() == 5;
-    }
-
-    @Override
-    public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-      CompactionPlan plan = new CompactionPlan();
-      plan.inputFiles.addAll(request.getFiles().keySet());
-      plan.writeParameters = new WriteParameters();
-      plan.writeParameters.setBlockSize(1024 * 1024);
-      plan.writeParameters.setCompressType("none");
-      plan.writeParameters.setHdfsBlockSize(1024 * 1024);
-      plan.writeParameters.setIndexBlockSize(10);
-      plan.writeParameters.setReplication(7);
-      return plan;
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      String tableName = getUniqueNames(1)[0];
-      client.tableOperations().create(tableName);
-      client.tableOperations().setProperty(tableName, Property.TABLE_COMPACTION_STRATEGY.getKey(),
-          TestCompactionStrategy.class.getName());
-      writeFile(client, tableName);
-      writeFile(client, tableName);
-      writeFile(client, tableName);
-      writeFile(client, tableName);
-      UtilWaitThread.sleep(2_000);
-      assertEquals(4, countFiles(client));
-      writeFile(client, tableName);
-      int count = countFiles(client);
-      assertTrue(count == 1 || count == 5);
-      while (count != 1) {
-        UtilWaitThread.sleep(250);
-        count = countFiles(client);
-      }
-    }
-  }
-
-  private int countFiles(AccumuloClient client) throws Exception {
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.setRange(TabletsSection.getRange());
-      s.fetchColumnFamily(DataFileColumnFamily.NAME);
-      return Iterators.size(s.iterator());
-    }
-  }
-
-  private void writeFile(AccumuloClient client, String tableName) throws Exception {
-    try (BatchWriter bw = client.createBatchWriter(tableName)) {
-      Mutation m = new Mutation("row");
-      m.put("cf", "cq", "value");
-      bw.addMutation(m);
-    }
-    client.tableOperations().flush(tableName, null, null, true);
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java
index 29e6d64..dd85425 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java
@@ -79,6 +79,7 @@
         TabletMetadata tm = tms.iterator().next();
         assertEquals(50, tm.getFiles().size());
       }
+
       IteratorSetting setting = new IteratorSetting(50, "ageoff", AgeOffFilter.class);
       setting.addOption("ttl", "0");
       setting.addOption("currentTime", Long.toString(System.currentTimeMillis() + 86400));
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java
index 459fc8b..ff54253 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java
@@ -41,6 +41,7 @@
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.core.spi.metrics.LoggingMeterRegistryFactory;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
@@ -70,8 +71,11 @@
       // Tell the server processes to use a StatsDMeterRegistry that will be configured
       // to push all metrics to the sink we started.
       cfg.setProperty(Property.GENERAL_MICROMETER_ENABLED, "true");
-      cfg.setProperty(Property.GENERAL_MICROMETER_FACTORY,
-          TestStatsDRegistryFactory.class.getName());
+      cfg.setProperty("general.custom.metrics.opts.logging.step", "5s");
+      String clazzList = LoggingMeterRegistryFactory.class.getName() + ","
+          + TestStatsDRegistryFactory.class.getName();
+      cfg.setProperty(Property.GENERAL_MICROMETER_FACTORY, clazzList);
+
       Map<String,String> sysProps = Map.of(TestStatsDRegistryFactory.SERVER_HOST, "127.0.0.1",
           TestStatsDRegistryFactory.SERVER_PORT, Integer.toString(sink.getPort()));
       cfg.setSystemProperties(sysProps);
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java
index abdf66b..e5724ea 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionProgressIT.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.test.compaction;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE1;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.compact;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.createTable;
@@ -32,7 +31,6 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.compactor.Compactor;
@@ -113,7 +111,12 @@
       try {
         while (!compactionFinished.get()) {
           checkRunning();
-          sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ex) {
+            log.debug("interrupted during sleep, forcing compaction finished as completed");
+            compactionFinished.set(true);
+          }
         }
       } catch (TException e) {
         log.warn("{}", e.getMessage(), e);
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
index 8719c88..63fa29e 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.compaction;
 
-import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.TabletClientHandler;
 import org.apache.accumulo.tserver.TabletServer;
@@ -26,7 +26,7 @@
 
 public class ExternalCompactionTServer extends TabletServer {
 
-  ExternalCompactionTServer(ServerOpts opts, String[] args) {
+  ExternalCompactionTServer(ConfigOpts opts, String[] args) {
     super(opts, args);
   }
 
@@ -38,7 +38,7 @@
 
   public static void main(String[] args) throws Exception {
     try (
-        ExternalCompactionTServer tserver = new ExternalCompactionTServer(new ServerOpts(), args)) {
+        ExternalCompactionTServer tserver = new ExternalCompactionTServer(new ConfigOpts(), args)) {
       tserver.runServer();
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
index 2d37f75..de24258 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTestUtils.java
@@ -63,7 +63,6 @@
 import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -76,6 +75,7 @@
 import org.apache.thrift.transport.TTransportException;
 
 import com.beust.jcommander.internal.Maps;
+import com.google.common.net.HostAndPort;
 
 public class ExternalCompactionTestUtils {
 
@@ -197,37 +197,37 @@
     clProps.put(ClientProperty.BATCH_WRITER_LATENCY_MAX.getKey(), "2s");
     cfg.setClientProps(clProps);
 
-    cfg.setProperty("tserver.compaction.major.service.cs1.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs1.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs1.planner.opts.executors",
         "[{'name':'all', 'type': 'external', 'queue': '" + QUEUE1 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs2.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs2.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs2.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs2.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE2 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs3.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs3.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs3.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs3.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE3 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs4.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs4.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs4.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs4.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE4 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs5.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs5.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs5.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs5.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE5 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs6.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs6.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs6.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs6.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE6 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs7.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs7.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs7.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs7.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE7 + "'}]");
-    cfg.setProperty("tserver.compaction.major.service.cs8.planner",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs8.planner",
         DefaultCompactionPlanner.class.getName());
-    cfg.setProperty("tserver.compaction.major.service.cs8.planner.opts.executors",
+    cfg.setProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "cs8.planner.opts.executors",
         "[{'name':'all', 'type': 'external','queue': '" + QUEUE8 + "'}]");
     cfg.setProperty(Property.COMPACTION_COORDINATOR_FINALIZER_COMPLETION_CHECK_INTERVAL, "5s");
     cfg.setProperty(Property.COMPACTION_COORDINATOR_DEAD_COMPACTOR_CHECK_INTERVAL, "5s");
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java
index 3f1b63c..a1a85e7 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.compaction;
 
+import static com.google.common.collect.MoreCollectors.onlyElement;
 import static org.apache.accumulo.minicluster.ServerType.TABLET_SERVER;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.MAX_DATA;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.QUEUE1;
@@ -34,6 +35,8 @@
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.row;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.verify;
 import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.writeData;
+import static org.apache.accumulo.test.util.FileMetadataUtil.countFencedFiles;
+import static org.apache.accumulo.test.util.FileMetadataUtil.splitFilesIntoRanges;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -46,6 +49,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
@@ -69,6 +73,7 @@
 import org.apache.accumulo.core.conf.Property;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.DevNull;
@@ -76,6 +81,7 @@
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState.FinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
@@ -404,23 +410,20 @@
       }
 
       LOG.info("Validating metadata table contents.");
-      TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets().forTable(tid)
-          .fetch(ColumnType.ECOMP).build();
-      List<TabletMetadata> md = new ArrayList<>();
-      tm.forEach(t -> md.add(t));
-      assertEquals(1, md.size());
-      TabletMetadata m = md.get(0);
-      Map<ExternalCompactionId,ExternalCompactionMetadata> em = m.getExternalCompactions();
-      assertEquals(1, em.size());
-      List<ExternalCompactionFinalState> finished = new ArrayList<>();
-      getFinalStatesForTable(getCluster(), tid).forEach(f -> finished.add(f));
-      assertEquals(1, finished.size());
-      assertEquals(em.entrySet().iterator().next().getKey(),
-          finished.get(0).getExternalCompactionId());
-      tm.close();
+      try (TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets()
+          .forTable(tid).fetch(ColumnType.ECOMP).build()) {
+        TabletMetadata m = tm.stream().collect(onlyElement());
+        Map<ExternalCompactionId,ExternalCompactionMetadata> em = m.getExternalCompactions();
+        assertEquals(1, em.size());
+        List<ExternalCompactionFinalState> finished = new ArrayList<>();
+        getFinalStatesForTable(getCluster(), tid).forEach(f -> finished.add(f));
+        assertEquals(1, finished.size());
+        assertEquals(em.entrySet().iterator().next().getKey(),
+            finished.get(0).getExternalCompactionId());
+      }
 
       // Force a flush on the metadata table before killing our tserver
-      client.tableOperations().flush("accumulo.metadata");
+      client.tableOperations().flush(AccumuloTable.METADATA.tableName());
 
       // Stop our TabletServer. Need to perform a normal shutdown so that the WAL is closed
       // normally.
@@ -449,6 +452,62 @@
     }
   }
 
+  @Test
+  public void testExternalCompactionWithFencedFiles() throws Exception {
+    String[] names = this.getUniqueNames(2);
+    try (AccumuloClient client =
+        Accumulo.newClient().from(getCluster().getClientProperties()).build()) {
+
+      String table1 = names[0];
+      createTable(client, table1, "cs1");
+
+      String table2 = names[1];
+      createTable(client, table2, "cs2");
+
+      writeData(client, table1);
+      writeData(client, table2);
+
+      // Verify that all data can be seen
+      verify(client, table1, 1, MAX_DATA);
+      verify(client, table2, 1, MAX_DATA);
+
+      // Split file in table1 into two files each fenced off by 100 rows for a total of 200
+      splitFilesIntoRanges(getCluster().getServerContext(), table1,
+          Set.of(new Range(new Text(row(100)), new Text(row(199))),
+              new Range(new Text(row(300)), new Text(row(399)))));
+      assertEquals(2, countFencedFiles(getCluster().getServerContext(), table1));
+
+      // Fence file in table2 to 600 rows
+      splitFilesIntoRanges(getCluster().getServerContext(), table2,
+          Set.of(new Range(new Text(row(200)), new Text(row(799)))));
+      assertEquals(1, countFencedFiles(getCluster().getServerContext(), table2));
+
+      // Verify that a subset of the data is now seen after fencing
+      verify(client, table1, 1, 200);
+      verify(client, table2, 1, 600);
+
+      getCluster().getClusterControl().startCoordinator(CompactionCoordinator.class);
+      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE1);
+      getCluster().getClusterControl().startCompactors(Compactor.class, 1, QUEUE2);
+
+      // Compact and verify previousy fenced data didn't come back
+      compact(client, table1, 2, QUEUE1, true);
+      verify(client, table1, 2, 200);
+
+      SortedSet<Text> splits = new TreeSet<>();
+      splits.add(new Text(row(MAX_DATA / 2)));
+      client.tableOperations().addSplits(table2, splits);
+
+      // Compact and verify previousy fenced data didn't come back
+      compact(client, table2, 3, QUEUE2, true);
+      verify(client, table2, 3, 600);
+
+      // should be no more fenced files after compaction
+      assertEquals(0, countFencedFiles(getCluster().getServerContext(), table1));
+      assertEquals(0, countFencedFiles(getCluster().getServerContext(), table2));
+    }
+  }
+
   public static class FSelector implements CompactionSelector {
 
     @Override
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java
index 9b0a94e..3b902a5 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java
@@ -336,10 +336,10 @@
       confirmCompactionCompleted(getCluster().getServerContext(), ecids,
           TCompactionState.CANCELLED);
 
-      TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets().forTable(tid)
-          .fetch(ColumnType.ECOMP).build();
-      assertEquals(0, tm.stream().count());
-      tm.close();
+      try (TabletsMetadata tm = getCluster().getServerContext().getAmple().readTablets()
+          .forTable(tid).fetch(ColumnType.ECOMP).build()) {
+        assertEquals(0, tm.stream().count());
+      }
 
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java
index 6b45472..c1f4d6f 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java
@@ -26,6 +26,7 @@
 import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.accumulo.compactor.Compactor;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface;
 import org.apache.accumulo.core.compaction.thrift.TCompactionState;
 import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate;
@@ -40,7 +41,7 @@
 
   private static final Logger LOG = LoggerFactory.getLogger(ExternalDoNothingCompactor.class);
 
-  ExternalDoNothingCompactor(CompactorServerOpts opts, String[] args) {
+  ExternalDoNothingCompactor(ConfigOpts opts, String[] args) {
     super(opts, args);
   }
 
@@ -90,7 +91,7 @@
   }
 
   public static void main(String[] args) throws Exception {
-    try (var compactor = new ExternalDoNothingCompactor(new CompactorServerOpts(), args)) {
+    try (var compactor = new ExternalDoNothingCompactor(new ConfigOpts(), args)) {
       compactor.runServer();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java b/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java
index f878165..4e8adc3 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/NonCommittingExternalCompactionTabletClientHandler.java
@@ -18,11 +18,13 @@
  */
 package org.apache.accumulo.test.compaction;
 
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.TabletClientHandler;
 import org.apache.accumulo.tserver.TabletServer;
@@ -30,7 +32,8 @@
 import org.apache.thrift.TException;
 
 public class NonCommittingExternalCompactionTabletClientHandler extends TabletClientHandler
-    implements TabletClientService.Iface {
+    implements TabletServerClientService.Iface, TabletIngestClientService.Iface,
+    TabletManagementClientService.Iface {
 
   public NonCommittingExternalCompactionTabletClientHandler(TabletServer server,
       TransactionWatcher watcher, WriteTracker writeTracker) {
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/SizeCompactionStrategy.java b/test/src/main/java/org/apache/accumulo/test/compaction/SizeCompactionStrategy.java
deleted file mode 100644
index b170c20..0000000
--- a/test/src/main/java/org/apache/accumulo/test/compaction/SizeCompactionStrategy.java
+++ /dev/null
@@ -1,65 +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.test.compaction;
-
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.CompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-
-@SuppressWarnings("removal")
-public class SizeCompactionStrategy extends CompactionStrategy {
-
-  private long size = 0;
-
-  @Override
-  public void init(Map<String,String> options) {
-    size = Long.parseLong(options.get("size"));
-  }
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-
-    for (DataFileValue dfv : request.getFiles().values()) {
-      if (dfv.getSize() < size) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-    CompactionPlan plan = new CompactionPlan();
-
-    for (Entry<StoredTabletFile,DataFileValue> entry : request.getFiles().entrySet()) {
-      if (entry.getValue().getSize() < size) {
-        plan.inputFiles.add(entry.getKey());
-      }
-    }
-
-    return plan;
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java
index af1e14d..63c00cb 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/SplitCancelsMajCIT.java
@@ -18,14 +18,13 @@
  */
 package org.apache.accumulo.test.compaction;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.EnumSet;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -89,7 +88,7 @@
       thread.start();
 
       long now = System.currentTimeMillis();
-      sleepUninterruptibly(10, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(10));
       // split the table, interrupts the compaction
       SortedSet<Text> partitionKeys = new TreeSet<>();
       partitionKeys.add(new Text("10"));
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java
index 7e5242b..8788d43 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java
@@ -23,13 +23,13 @@
 
 import org.apache.accumulo.coordinator.CompactionCoordinator;
 import org.apache.accumulo.coordinator.CompactionFinalizer;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState.FinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,7 +62,7 @@
 
   }
 
-  protected TestCompactionCoordinatorForOfflineTable(ServerOpts opts, String[] args) {
+  protected TestCompactionCoordinatorForOfflineTable(ConfigOpts opts, String[] args) {
     super(opts, args);
   }
 
@@ -73,7 +73,7 @@
 
   public static void main(String[] args) throws Exception {
     try (TestCompactionCoordinatorForOfflineTable coordinator =
-        new TestCompactionCoordinatorForOfflineTable(new ServerOpts(), args)) {
+        new TestCompactionCoordinatorForOfflineTable(new ConfigOpts(), args)) {
       coordinator.runServer();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionStrategy.java b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionStrategy.java
deleted file mode 100644
index 7937f06..0000000
--- a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionStrategy.java
+++ /dev/null
@@ -1,81 +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.test.compaction;
-
-import java.util.Map;
-
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.CompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-
-@SuppressWarnings("removal")
-public class TestCompactionStrategy extends CompactionStrategy {
-
-  private String inputPrefix = "Z";
-  private String dropPrefix = "Z";
-  private boolean shouldCompact = false;
-
-  @Override
-  public void init(Map<String,String> options) {
-    if (options.containsKey("inputPrefix")) {
-      inputPrefix = options.get("inputPrefix");
-    }
-    if (options.containsKey("dropPrefix")) {
-      dropPrefix = options.get("dropPrefix");
-    }
-    if (options.containsKey("shouldCompact")) {
-      shouldCompact = Boolean.parseBoolean(options.get("shouldCompact"));
-    }
-  }
-
-  @Override
-  public boolean shouldCompact(MajorCompactionRequest request) {
-    if (shouldCompact) {
-      return true;
-    }
-
-    for (TabletFile file : request.getFiles().keySet()) {
-      if (file.getFileName().startsWith(inputPrefix)) {
-        return true;
-      }
-      if (file.getFileName().startsWith(dropPrefix)) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  @Override
-  public CompactionPlan getCompactionPlan(MajorCompactionRequest request) {
-    CompactionPlan plan = new CompactionPlan();
-
-    for (StoredTabletFile file : request.getFiles().keySet()) {
-      if (file.getFileName().startsWith(dropPrefix)) {
-        plan.deleteFiles.add(file);
-      } else if (file.getFileName().startsWith(inputPrefix)) {
-        plan.inputFiles.add(file);
-      }
-    }
-
-    return plan;
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/UserCompactionStrategyIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/UserCompactionStrategyIT.java
deleted file mode 100644
index 4f1a537..0000000
--- a/test/src/main/java/org/apache/accumulo/test/compaction/UserCompactionStrategyIT.java
+++ /dev/null
@@ -1,334 +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.test.compaction;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-import static org.junit.jupiter.api.Assumptions.assumeTrue;
-
-import java.io.File;
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.CompactionConfig;
-import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.conf.Property;
-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.iterators.user.RegExFilter;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.test.functional.FunctionalTestUtils;
-import org.apache.accumulo.test.functional.SlowIterator;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("removal")
-public class UserCompactionStrategyIT extends AccumuloClusterHarness {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(3);
-  }
-
-  @AfterEach
-  public void checkForDanglingFateLocks() {
-    if (getClusterType() == ClusterType.MINI) {
-      FunctionalTestUtils.assertNoDanglingFateLocks(getCluster());
-    }
-  }
-
-  @Test
-  public void testDropA() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-
-      writeFlush(c, tableName, "a");
-      writeFlush(c, tableName, "b");
-      // create a file that starts with A containing rows 'a' and 'b'
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-
-      writeFlush(c, tableName, "c");
-      writeFlush(c, tableName, "d");
-
-      // drop files that start with A
-      CompactionStrategyConfig csConfig =
-          new CompactionStrategyConfig(TestCompactionStrategy.class.getName());
-      csConfig.setOptions(Map.of("dropPrefix", "A", "inputPrefix", "F"));
-      c.tableOperations().compact(tableName,
-          new CompactionConfig().setWait(true).setCompactionStrategy(csConfig));
-
-      assertEquals(Set.of("c", "d"), getRows(c, tableName));
-
-      // this compaction should not drop files starting with A
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-
-      assertEquals(Set.of("c", "d"), getRows(c, tableName));
-    }
-  }
-
-  private void testDropNone(Map<String,String> options) throws Exception {
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-
-      writeFlush(c, tableName, "a");
-      writeFlush(c, tableName, "b");
-
-      CompactionStrategyConfig csConfig =
-          new CompactionStrategyConfig(TestCompactionStrategy.class.getName());
-      csConfig.setOptions(options);
-      c.tableOperations().compact(tableName,
-          new CompactionConfig().setWait(true).setCompactionStrategy(csConfig));
-
-      assertEquals(Set.of("a", "b"), getRows(c, tableName));
-    }
-  }
-
-  @Test
-  public void testDropNone() throws Exception {
-    // test a compaction strategy that selects no files. In this case there is no work to do, want
-    // to ensure it does not hang.
-
-    testDropNone(Map.of("inputPrefix", "Z"));
-  }
-
-  @Test
-  public void testDropNone2() throws Exception {
-    // test a compaction strategy that selects no files. This differs testDropNone() in that
-    // shouldCompact() will return true and getCompactionPlan() will
-    // return no work to do.
-
-    testDropNone(Map.of("inputPrefix", "Z", "shouldCompact", "true"));
-  }
-
-  @Test
-  public void testPerTableClasspath() throws Exception {
-    // Can't assume that a test-resource will be on the server's classpath
-    assumeTrue(getClusterType() == ClusterType.MINI);
-
-    // test per-table classpath + user specified compaction strategy
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      final String tableName = getUniqueNames(1)[0];
-      File target = new File(System.getProperty("user.dir"), "target");
-      assertTrue(target.mkdirs() || target.isDirectory());
-      var destFile = initJar("/org/apache/accumulo/test/TestCompactionStrat.jar",
-          "TestCompactionStrat", target.getAbsolutePath());
-      c.instanceOperations().setProperty(
-          Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "context1", destFile.toString());
-      HashMap<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_CLASSLOADER_CONTEXT.getKey(), "context1");
-      SortedSet<Text> splits = new TreeSet<>(Arrays.asList(new Text("efg")));
-      var ntc = new NewTableConfiguration().setProperties(props).withSplits(splits);
-      c.tableOperations().create(tableName, ntc);
-
-      writeFlush(c, tableName, "a");
-      writeFlush(c, tableName, "b");
-
-      writeFlush(c, tableName, "h");
-      writeFlush(c, tableName, "i");
-
-      assertEquals(4, FunctionalTestUtils.countRFiles(c, tableName));
-
-      // EfgCompactionStrat will only compact a tablet w/ end row of 'efg'. No other tablets are
-      // compacted.
-      CompactionStrategyConfig csConfig =
-          new CompactionStrategyConfig("org.apache.accumulo.test.EfgCompactionStrat");
-      c.tableOperations().compact(tableName,
-          new CompactionConfig().setWait(true).setCompactionStrategy(csConfig));
-
-      assertEquals(3, FunctionalTestUtils.countRFiles(c, tableName));
-
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-
-      assertEquals(2, FunctionalTestUtils.countRFiles(c, tableName));
-    }
-  }
-
-  @Test
-  public void testIterators() throws Exception {
-    // test compaction strategy + iterators
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-
-      writeFlush(c, tableName, "a");
-      writeFlush(c, tableName, "b");
-      // create a file that starts with A containing rows 'a' and 'b'
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-
-      writeFlush(c, tableName, "c");
-      writeFlush(c, tableName, "d");
-
-      assertEquals(3, FunctionalTestUtils.countRFiles(c, tableName));
-
-      // drop files that start with A
-      CompactionStrategyConfig csConfig =
-          new CompactionStrategyConfig(TestCompactionStrategy.class.getName());
-      csConfig.setOptions(Map.of("inputPrefix", "F"));
-
-      IteratorSetting iterConf = new IteratorSetting(21, "myregex", RegExFilter.class);
-      RegExFilter.setRegexs(iterConf, "a|c", null, null, null, false);
-
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true)
-          .setCompactionStrategy(csConfig).setIterators(Arrays.asList(iterConf)));
-
-      // compaction strategy should only be applied to one file. If its applied to both, then row
-      // 'b'
-      // would be dropped by filter.
-      assertEquals(Set.of("a", "b", "c"), getRows(c, tableName));
-
-      assertEquals(2, FunctionalTestUtils.countRFiles(c, tableName));
-
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-
-      // ensure that iterator is not applied
-      assertEquals(Set.of("a", "b", "c"), getRows(c, tableName));
-
-      assertEquals(1, FunctionalTestUtils.countRFiles(c, tableName));
-    }
-  }
-
-  @Test
-  public void testFileSize() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-
-      // write random data because its very unlikely it will compress
-      writeRandomValue(c, tableName, 1 << 16);
-      writeRandomValue(c, tableName, 1 << 16);
-
-      writeRandomValue(c, tableName, 1 << 9);
-      writeRandomValue(c, tableName, 1 << 7);
-      writeRandomValue(c, tableName, 1 << 6);
-
-      assertEquals(5, FunctionalTestUtils.countRFiles(c, tableName));
-
-      CompactionStrategyConfig csConfig =
-          new CompactionStrategyConfig(SizeCompactionStrategy.class.getName());
-      csConfig.setOptions(Map.of("size", "" + (1 << 15)));
-      c.tableOperations().compact(tableName,
-          new CompactionConfig().setWait(true).setCompactionStrategy(csConfig));
-
-      assertEquals(3, FunctionalTestUtils.countRFiles(c, tableName));
-
-      csConfig = new CompactionStrategyConfig(SizeCompactionStrategy.class.getName());
-      csConfig.setOptions(Map.of("size", "" + (1 << 17)));
-      c.tableOperations().compact(tableName,
-          new CompactionConfig().setWait(true).setCompactionStrategy(csConfig));
-
-      assertEquals(1, FunctionalTestUtils.countRFiles(c, tableName));
-    }
-
-  }
-
-  @Test
-  public void testConcurrent() throws Exception {
-    // two compactions without iterators or strategy should be able to run concurrently
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-
-      // write random data because its very unlikely it will compress
-      writeRandomValue(c, tableName, 1 << 16);
-      writeRandomValue(c, tableName, 1 << 16);
-
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(false));
-      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-
-      assertEquals(1, FunctionalTestUtils.countRFiles(c, tableName));
-
-      writeRandomValue(c, tableName, 1 << 16);
-
-      IteratorSetting iterConfig = new IteratorSetting(30, SlowIterator.class);
-      SlowIterator.setSleepTime(iterConfig, 1000);
-
-      long t1 = System.currentTimeMillis();
-      c.tableOperations().compact(tableName,
-          new CompactionConfig().setWait(false).setIterators(Arrays.asList(iterConfig)));
-      try {
-        // this compaction should fail because previous one set iterators
-        c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
-        if (System.currentTimeMillis() - t1 < 2000) {
-          fail("Expected compaction to fail because another concurrent compaction set iterators");
-        }
-      } catch (AccumuloException e) {}
-    }
-  }
-
-  void writeRandomValue(AccumuloClient c, String tableName, int size) throws Exception {
-    byte[] data1 = new byte[size];
-    random.nextBytes(data1);
-
-    try (BatchWriter bw = c.createBatchWriter(tableName)) {
-      Mutation m1 = new Mutation("r" + random.nextInt(909090));
-      m1.put("data", "bl0b", new Value(data1));
-      bw.addMutation(m1);
-    }
-    c.tableOperations().flush(tableName, null, null, true);
-  }
-
-  private Set<String> getRows(AccumuloClient c, String tableName) throws TableNotFoundException {
-    Set<String> rows = new HashSet<>();
-    try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
-      for (Entry<Key,Value> entry : scanner) {
-        rows.add(entry.getKey().getRowData().toString());
-      }
-    }
-    return rows;
-  }
-
-  private void writeFlush(AccumuloClient client, String tablename, String row) throws Exception {
-    try (BatchWriter bw = client.createBatchWriter(tablename)) {
-      Mutation m = new Mutation(row);
-      m.put("", "", "");
-      bw.addMutation(m);
-    }
-    client.tableOperations().flush(tablename, null, null, true);
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java b/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java
index e80ffdd..0fb40ab 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/PropStoreConfigIT.java
@@ -21,7 +21,6 @@
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.conf.ConfigurationTypeHelper.getMemoryAsBytes;
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -65,7 +64,6 @@
 import org.slf4j.LoggerFactory;
 
 @Tag(MINI_CLUSTER_ONLY)
-@Tag(SUNNY_DAY)
 public class PropStoreConfigIT extends SharedMiniClusterBase {
 
   private static final Logger log = LoggerFactory.getLogger(PropStoreConfigIT.class);
@@ -333,6 +331,47 @@
   }
 
   @Test
+  public void getSystemPropertiesTest() throws Exception {
+
+    // Tests that getSystemProperties() does not return a merged view (only returns props set at the
+    // system level).
+    // Compares this with getSystemConfiguration() which does return a merged view (system + site +
+    // default).
+
+    String customPropKey1 = "table.custom.prop1";
+    String customPropKey2 = "table.custom.prop2";
+    String customPropVal1 = "v1";
+    String customPropVal2 = "v2";
+    try (var client = Accumulo.newClient().from(getClientProps()).build()) {
+      // non-merged view
+      Map<String,String> sysProps = client.instanceOperations().getSystemProperties();
+      // merged view
+      Map<String,String> sysConfig = client.instanceOperations().getSystemConfiguration();
+      // sysProps is non-merged view, so should be empty at this point (no system props set yet)
+      assertTrue(sysProps.isEmpty());
+      // sysConfig is merged view, so will have some props already (default properties and those
+      // inherited from site config)
+      assertFalse(sysConfig.isEmpty());
+      client.instanceOperations().setProperty(customPropKey1, customPropVal1);
+      client.instanceOperations().setProperty(customPropKey2, customPropVal2);
+      Wait.waitFor(() -> client.instanceOperations().getSystemConfiguration().get(customPropKey1)
+          .equals(customPropVal1), 5000, 500);
+      Wait.waitFor(() -> client.instanceOperations().getSystemConfiguration().get(customPropKey2)
+          .equals(customPropVal2), 5000, 500);
+      sysProps = client.instanceOperations().getSystemProperties();
+      // The custom props should be present (and the only props) in the non-merged view
+      assertEquals(2, sysProps.size());
+      assertEquals(customPropVal1, sysProps.get(customPropKey1));
+      assertEquals(customPropVal2, sysProps.get(customPropKey2));
+      sysConfig = client.instanceOperations().getSystemConfiguration();
+      // The custom props should be present in the merged view
+      assertTrue(sysConfig.size() > 2);
+      assertEquals(customPropVal1, sysConfig.get(customPropKey1));
+      assertEquals(customPropVal2, sysConfig.get(customPropKey2));
+    }
+  }
+
+  @Test
   public void modifyTablePropTest() throws Exception {
     String table = getUniqueNames(1)[0];
 
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java b/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java
deleted file mode 100644
index a3a59ca..0000000
--- a/test/src/main/java/org/apache/accumulo/test/conf/util/ConfigTransformerIT.java
+++ /dev/null
@@ -1,187 +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.test.conf.util;
-
-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.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import java.io.File;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.util.Retry;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
-import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
-import org.apache.accumulo.server.conf.util.ConfigTransformer;
-import org.apache.accumulo.server.conf.util.TransformToken;
-import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Tag(ZOOKEEPER_TESTING_SERVER)
-public class ConfigTransformerIT {
-
-  private static final Logger log = LoggerFactory.getLogger(ConfigTransformerIT.class);
-  private final static VersionedPropCodec codec = VersionedPropCodec.getDefault();
-  @TempDir
-  private static File tempDir;
-  private static ZooKeeperTestingServer testZk = null;
-  private static ZooKeeper zooKeeper;
-  private static ZooReaderWriter zrw;
-
-  private InstanceId instanceId = null;
-  private ZooPropStore propStore = null;
-  private ServerContext context = null;
-  private PropStoreWatcher watcher = null;
-
-  @BeforeAll
-  public static void setupZk() {
-
-    // using default zookeeper port - we don't have a full configuration
-    testZk = new ZooKeeperTestingServer(tempDir);
-    zooKeeper = testZk.getZooKeeper();
-    ZooUtil.digestAuth(zooKeeper, ZooKeeperTestingServer.SECRET);
-    zrw = testZk.getZooReaderWriter();
-  }
-
-  @AfterAll
-  public static void shutdownZK() throws Exception {
-    testZk.close();
-  }
-
-  @BeforeEach
-  public void testSetup() throws Exception {
-    instanceId = InstanceId.of(UUID.randomUUID());
-
-    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
-    for (LegacyPropData.PropNode node : nodes) {
-      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
-    }
-    propStore = ZooPropStore.initialize(instanceId, zrw);
-
-    context = createMock(ServerContext.class);
-    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
-    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
-    expect(context.getPropStore()).andReturn(propStore).anyTimes();
-
-    watcher = createMock(PropStoreWatcher.class);
-    watcher.process(anyObject());
-    expectLastCall().anyTimes();
-
-    replay(context, watcher);
-
-  }
-
-  @AfterEach
-  public void cleanupZnodes() {
-    try {
-      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
-    } catch (KeeperException | InterruptedException ex) {
-      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
-    }
-    verify(context, watcher);
-  }
-
-  @Test
-  public void propStoreConversionTest() throws Exception {
-
-    var sysPropKey = SystemPropKey.of(instanceId);
-
-    List<String> sysLegacy = zrw.getChildren(sysPropKey.getPath());
-    log.info("Before: {}", sysLegacy);
-
-    var vProps = propStore.get(sysPropKey);
-    assertNotNull(vProps);
-    log.info("Converted: {}", vProps);
-
-    sysLegacy = zrw.getChildren(sysPropKey.getPath());
-    log.info("After: {}", sysLegacy);
-
-  }
-
-  @Test
-  public void transformTest() throws Exception {
-
-    var sysPropKey = SystemPropKey.of(instanceId);
-
-    ConfigTransformer transformer = new ConfigTransformer(zrw, codec, watcher);
-    List<String> sysLegacy = zrw.getChildren(sysPropKey.getPath());
-    log.info("Before: {}", sysLegacy);
-
-    var converted = transformer.transform(sysPropKey, sysPropKey.getPath(), false);
-
-    assertEquals(sysLegacy.size(), converted.asMap().size());
-  }
-
-  @Test
-  public void failToGetLock() throws Exception {
-    var sysPropKey = SystemPropKey.of(instanceId);
-
-    Retry retry =
-        Retry.builder().maxRetries(3).retryAfter(250, MILLISECONDS).incrementBy(500, MILLISECONDS)
-            .maxWait(5, SECONDS).backOffFactor(1.75).logInterval(3, MINUTES).createRetry();
-
-    ConfigTransformer transformer = new ConfigTransformer(zrw, codec, watcher, retry);
-    // manually create a lock so transformer fails
-    zrw.putEphemeralData(sysPropKey.getPath() + TransformToken.TRANSFORM_TOKEN, new byte[0]);
-
-    assertThrows(IllegalStateException.class,
-        () -> transformer.transform(sysPropKey, sysPropKey.getPath(), false));
-
-  }
-
-  @Test
-  public void continueOnLockRelease() {
-
-  }
-
-  @Test
-  public void createdByAnother() {
-
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/LegacyPropData.java b/test/src/main/java/org/apache/accumulo/test/conf/util/LegacyPropData.java
deleted file mode 100644
index 19bb1cb..0000000
--- a/test/src/main/java/org/apache/accumulo/test/conf/util/LegacyPropData.java
+++ /dev/null
@@ -1,390 +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.test.conf.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-
-/**
- * Provides sampled ZooKeeper properties paths / data from a 1.10 instance for testing.
- */
-public class LegacyPropData {
-
-  /**
-   * Generates a list of ZooKeeper nodes captured from a 1.10 instance. The list is ordered so that
-   * parent nodes are created before the children. This is used for property conversion testing and
-   * other values are irrelevant and left as empty ZooKeeper nodes.
-   */
-  public static List<PropNode> getData(final InstanceId instanceId) {
-    String zkRoot = ZooUtil.getRoot(instanceId);
-    List<PropNode> names = new ArrayList<>(250);
-
-    names.add(new PropNode(Constants.ZROOT, null));
-    names.add(new PropNode(ZooUtil.getRoot(instanceId), null));
-
-    names.add(new PropNode(zkRoot + "/bulk_failed_copyq", null));
-    names.add(new PropNode(zkRoot + "/config", null));
-    names.add(new PropNode(zkRoot + "/dead", null));
-    names.add(new PropNode(zkRoot + "/fate", null));
-    names.add(new PropNode(zkRoot + "/gc", null));
-    names.add(new PropNode(zkRoot + "/hdfs_reservations", null));
-    names.add(new PropNode(zkRoot + "/masters", null));
-    names.add(new PropNode(zkRoot + "/monitor", null));
-    names.add(new PropNode(zkRoot + "/namespaces", null));
-    names.add(new PropNode(zkRoot + "/next_file", null));
-    names.add(new PropNode(zkRoot + "/problems", null));
-    names.add(new PropNode(zkRoot + "/recovery", null));
-    names.add(new PropNode(zkRoot + "/replication", null));
-    names.add(new PropNode(zkRoot + "/root_tablet", null));
-    names.add(new PropNode(zkRoot + "/table_locks", null));
-    names.add(new PropNode(zkRoot + "/tables", null));
-    names.add(new PropNode(zkRoot + "/tservers", null));
-    names.add(new PropNode(zkRoot + "/users", null));
-    names.add(new PropNode(zkRoot + "/wals", null));
-    names.add(new PropNode(zkRoot + "/bulk_failed_copyq/locks", null));
-    names.add(new PropNode(zkRoot + "/config/master.bulk.retries", "4"));
-    names.add(new PropNode(zkRoot + "/config/master.bulk.timeout", "10m"));
-    names.add(new PropNode(zkRoot + "/config/table.bloom.enabled", "true"));
-    names.add(new PropNode(zkRoot + "/config/master.bulk.rename.threadpool.size", "10"));
-    names.add(new PropNode(zkRoot + "/config/master.bulk.threadpool.size", "4"));
-    names.add(new PropNode(zkRoot + "/dead/tservers", null));
-    names.add(new PropNode(zkRoot + "/gc/lock", null));
-    names.add(new PropNode(zkRoot + "/gc/lock/zlock-0000000000", null));
-    names.add(new PropNode(zkRoot + "/masters/goal_state", null));
-    names.add(new PropNode(zkRoot + "/masters/lock", null));
-    names.add(new PropNode(zkRoot + "/masters/repl_coord_addr", null));
-    names.add(new PropNode(zkRoot + "/masters/tick", null));
-    names.add(new PropNode(zkRoot + "/masters/lock/zlock-0000000000", null));
-    names.add(new PropNode(zkRoot + "/monitor/http_addr", null));
-    names.add(new PropNode(zkRoot + "/monitor/lock", null));
-    names.add(new PropNode(zkRoot + "/monitor/log4j_addr", null));
-    names.add(new PropNode(zkRoot + "/monitor/lock/zlock-0000000000", null));
-    names.add(new PropNode(zkRoot + "/namespaces/+accumulo", null));
-    names.add(new PropNode(zkRoot + "/namespaces/+default", null));
-    names.add(new PropNode(zkRoot + "/namespaces/2", null));
-    names.add(new PropNode(zkRoot + "/namespaces/3", null));
-    names.add(new PropNode(zkRoot + "/namespaces/+accumulo/conf", null));
-    names.add(new PropNode(zkRoot + "/namespaces/+accumulo/name", "accumulo"));
-    names.add(new PropNode(zkRoot + "/namespaces/+default/conf", null));
-    names.add(new PropNode(zkRoot + "/namespaces/+default/name", null));
-    names.add(new PropNode(zkRoot + "/namespaces/2/conf", null));
-    names.add(new PropNode(zkRoot + "/namespaces/2/name", "ns1"));
-    names.add(new PropNode(zkRoot + "/namespaces/2/conf/table.bloom.enabled", "false"));
-    names.add(new PropNode(zkRoot + "/namespaces/3/conf", null));
-    names.add(new PropNode(zkRoot + "/namespaces/3/name", "ns2"));
-    names.add(new PropNode(zkRoot + "/recovery/locks", null));
-    names.add(new PropNode(zkRoot + "/replication/tservers", null));
-    names.add(new PropNode(zkRoot + "/replication/workqueue", null));
-    names.add(new PropNode(zkRoot + "/replication/tservers/localhost:11000", null));
-    names.add(new PropNode(zkRoot + "/replication/workqueue/locks", null));
-    names.add(new PropNode(zkRoot + "/root_tablet/current_logs", null));
-    names.add(new PropNode(zkRoot + "/root_tablet/dir", null));
-    names.add(new PropNode(zkRoot + "/root_tablet/lastlocation", null));
-    names.add(new PropNode(zkRoot + "/root_tablet/location", null));
-    names.add(new PropNode(zkRoot + "/root_tablet/walogs", null));
-    names.add(new PropNode(zkRoot + "/tables/!0", null));
-    names.add(new PropNode(zkRoot + "/tables/+r", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep", null));
-    names.add(new PropNode(zkRoot + "/tables/1", null));
-    names.add(new PropNode(zkRoot + "/tables/4", null));
-    names.add(new PropNode(zkRoot + "/tables/5", null));
-    names.add(new PropNode(zkRoot + "/tables/6", null));
-    names.add(new PropNode(zkRoot + "/tables/7", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/name", "metadata"));
-    names.add(new PropNode(zkRoot + "/tables/!0/namespace", "+accumulo"));
-    names.add(new PropNode(zkRoot + "/tables/!0/state", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.failures.ignore", "false"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.cache.index.enable", "true"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.group.server", "file,log,srv,future"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.scan.replcombiner.opt.columns",
-        "stat"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.compaction.major.ratio", "1"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.file.replication", "5"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.majc.replcombiner",
-        "9,org.apache.accumulo.server.replication.StatusCombiner"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.scan.replcombiner",
-        "9,org.apache.accumulo.server.replication.StatusCombiner"));
-    names.add(
-        new PropNode(zkRoot + "/tables/!0/conf/table.iterator.scan.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.minc.vers",
-        "10,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(
-        new PropNode(zkRoot + "/tables/!0/conf/table.iterator.majc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.scan.vers",
-        "10,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.group.tablet", "~tab,loc"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.groups.enabled", "tablet,server"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.majc.vers",
-        "10,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.constraint.1",
-        "org.apache.accumulo.server.constraints.MetadataConstraints"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.minc.replcombiner.opt.columns",
-        "stat"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.split.threshold", "64M"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.majc.bulkLoadFilter",
-        "20,org.apache.accumulo.server.iterators.MetadataBulkLoadFilter"));
-    names.add(
-        new PropNode(zkRoot + "/tables/!0/conf/table.iterator.minc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.minc.replcombiner",
-        "9,org.apache.accumulo.server.replication.StatusCombiner"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.file.compress.blocksize", "32K"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.durability", "sync"));
-    names
-        .add(new PropNode(zkRoot + "/tables/!0/conf/table.security.scan.visibility.default", null));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.cache.block.enable", "true"));
-    names.add(new PropNode(zkRoot + "/tables/!0/conf/table.iterator.majc.replcombiner.opt.columns",
-        "stat"));
-    names.add(new PropNode(zkRoot + "/tables/+r/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/+r/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/+r/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/+r/name", "root"));
-    names.add(new PropNode(zkRoot + "/tables/+r/namespace", "+accumulo"));
-    names.add(new PropNode(zkRoot + "/tables/+r/state", null));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.group.tablet", "~tab,loc"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.groups.enabled", "tablet,server"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.iterator.majc.vers",
-        "10,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.constraint.1",
-        "org.apache.accumulo.server.constraints.MetadataConstraints"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.split.threshold", "64M"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.failures.ignore", "false"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.iterator.majc.bulkLoadFilter",
-        "20,org.apache.accumulo.server.iterators.MetadataBulkLoadFilter"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.cache.index.enable", "true"));
-    names.add(
-        new PropNode(zkRoot + "/tables/+r/conf/table.iterator.minc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.group.server", "file,log,srv,future"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.file.compress.blocksize", "32K"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.durability", "sync"));
-    names
-        .add(new PropNode(zkRoot + "/tables/+r/conf/table.security.scan.visibility.default", null));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.compaction.major.ratio", "1"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.file.replication", "5"));
-    names.add(
-        new PropNode(zkRoot + "/tables/+r/conf/table.iterator.scan.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.iterator.minc.vers",
-        "10,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(
-        new PropNode(zkRoot + "/tables/+r/conf/table.iterator.majc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.cache.block.enable", "true"));
-    names.add(new PropNode(zkRoot + "/tables/+r/conf/table.iterator.scan.vers",
-        "10,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/name", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/namespace", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/state", null));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.iterator.minc.statuscombiner",
-        "30,org.apache.accumulo.server.replication.StatusCombiner"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.groups.enabled", "repl,work"));
-    names.add(new PropNode(
-        zkRoot + "/tables/+rep/conf/table.iterator.majc.statuscombiner.opt.columns", "repl,work"));
-    names.add(new PropNode(
-        zkRoot + "/tables/+rep/conf/table.iterator.minc.statuscombiner.opt.columns", "repl,work"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.group.work", "work"));
-    names.add(new PropNode(
-        zkRoot + "/tables/+rep/conf/table.iterator.scan.statuscombiner.opt.columns", "repl,work"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.iterator.majc.statuscombiner",
-        "30,org.apache.accumulo.server.replication.StatusCombiner"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.group.repl", "repl"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.iterator.scan.statuscombiner",
-        "30,org.apache.accumulo.server.replication.StatusCombiner"));
-    names.add(new PropNode(zkRoot + "/tables/+rep/conf/table.formatter",
-        "org.apache.accumulo.server.replication.StatusFormatter"));
-    names.add(new PropNode(zkRoot + "/tables/1/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/1/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/1/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/1/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/1/name", "trace"));
-    names.add(new PropNode(zkRoot + "/tables/1/namespace", "+default"));
-    names.add(new PropNode(zkRoot + "/tables/1/state", null));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.constraint.1",
-        "org.apache.accumulo.core.constraints.DefaultKeySizeConstraint"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.formatter",
-        "org.apache.accumulo.tracer.TraceFormatter"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.majc.ageoff",
-        "10,org.apache.accumulo.core.iterators.user.AgeOffFilter"));
-    names.add(
-        new PropNode(zkRoot + "/tables/1/conf/table.iterator.majc.ageoff.opt.ttl", "604800000"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.majc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.majc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.minc.ageoff",
-        "10,org.apache.accumulo.core.iterators.user.AgeOffFilter"));
-    names.add(
-        new PropNode(zkRoot + "/tables/1/conf/table.iterator.minc.ageoff.opt.ttl", "604800000"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.minc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.minc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.scan.ageoff",
-        "10,org.apache.accumulo.core.iterators.user.AgeOffFilter"));
-    names.add(
-        new PropNode(zkRoot + "/tables/1/conf/table.iterator.scan.ageoff.opt.ttl", "604800000"));
-    names.add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.scan.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/1/conf/table.iterator.scan.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/4/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/4/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/4/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/4/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/4/name", "tbl1"));
-    names.add(new PropNode(zkRoot + "/tables/4/namespace", "2"));
-    names.add(new PropNode(zkRoot + "/tables/4/state", null));
-    names.add(new PropNode(zkRoot + "/tables/4/conf/table.iterator.majc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(new PropNode(zkRoot + "/tables/4/conf/table.constraint.1",
-        "org.apache.accumulo.core.constraints.DefaultKeySizeConstraint"));
-    names
-        .add(new PropNode(zkRoot + "/tables/4/conf/table.iterator.scan.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/4/conf/table.iterator.minc.vers",
-        "/20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/4/conf/table.iterator.majc.vers.opt.maxVersions", "1"));
-    names
-        .add(new PropNode(zkRoot + "/tables/4/conf/table.iterator.minc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/4/conf/table.iterator.scan.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names.add(new PropNode(zkRoot + "/tables/4/conf/table.bloom.enabled", "true"));
-    names.add(new PropNode(zkRoot + "/tables/5/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/5/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/5/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/5/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/5/name", "tbl2"));
-    names.add(new PropNode(zkRoot + "/tables/5/namespace", "2"));
-    names.add(new PropNode(zkRoot + "/tables/5/state", null));
-    names.add(new PropNode(zkRoot + "/tables/5/conf/table.constraint.1",
-        "org.apache.accumulo.core.constraints.DefaultKeySizeConstraint"));
-    names.add(new PropNode(zkRoot + "/tables/5/conf/table.iterator.majc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/5/conf/table.iterator.majc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/5/conf/table.iterator.minc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/5/conf/table.iterator.minc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/5/conf/table.iterator.scan.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/5/conf/table.iterator.scan.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/6/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/6/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/6/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/6/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/6/name", "tbl3"));
-    names.add(new PropNode(zkRoot + "/tables/6/namespace", "+default"));
-    names.add(new PropNode(zkRoot + "/tables/6/state", null));
-    names.add(new PropNode(zkRoot + "/tables/6/conf/table.bloom.enabled", "true"));
-    names.add(new PropNode(zkRoot + "/tables/6/conf/table.constraint.1",
-        "org.apache.accumulo.core.constraints.DefaultKeySizeConstraint"));
-    names.add(new PropNode(zkRoot + "/tables/6/conf/table.iterator.majc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/6/conf/table.iterator.majc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/6/conf/table.iterator.minc.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/6/conf/table.iterator.minc.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/6/conf/table.iterator.scan.vers",
-        "20,org.apache.accumulo.core.iterators.user.VersioningIterator"));
-    names
-        .add(new PropNode(zkRoot + "/tables/6/conf/table.iterator.scan.vers.opt.maxVersions", "1"));
-    names.add(new PropNode(zkRoot + "/tables/7/compact-cancel-id", null));
-    names.add(new PropNode(zkRoot + "/tables/7/compact-id", null));
-    names.add(new PropNode(zkRoot + "/tables/7/conf", null));
-    names.add(new PropNode(zkRoot + "/tables/7/flush-id", null));
-    names.add(new PropNode(zkRoot + "/tables/7/name", "tbl4"));
-    names.add(new PropNode(zkRoot + "/tables/7/namespace", "2"));
-    names.add(new PropNode(zkRoot + "/tables/7/state", null));
-    names.add(new PropNode(zkRoot + "/tables/7/conf/table.bloom.enabled", null));
-    names.add(new PropNode(zkRoot + "/tables/7/conf/table.constraint.1", null));
-    names.add(new PropNode(zkRoot + "/tables/7/conf/table.iterator.majc.vers", null));
-    names.add(
-        new PropNode(zkRoot + "/tables/7/conf/table.iterator.majc.vers.opt.maxVersions", null));
-    names.add(new PropNode(zkRoot + "/tables/7/conf/table.iterator.minc.vers", null));
-    names.add(
-        new PropNode(zkRoot + "/tables/7/conf/table.iterator.minc.vers.opt.maxVersions", null));
-    names.add(new PropNode(zkRoot + "/tables/7/conf/table.iterator.scan.vers", null));
-    names.add(
-        new PropNode(zkRoot + "/tables/7/conf/table.iterator.scan.vers.opt.maxVersions", null));
-    names.add(new PropNode(zkRoot + "/tservers/localhost:11000", null));
-    names.add(new PropNode(zkRoot + "/tservers/localhost:11000/zlock-0000000000", null));
-    names.add(new PropNode(zkRoot + "/users/root", null));
-    names.add(new PropNode(zkRoot + "/users/root/Authorizations", null));
-    names.add(new PropNode(zkRoot + "/users/root/Namespaces", null));
-    names.add(new PropNode(zkRoot + "/users/root/System", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables", null));
-    names.add(new PropNode(zkRoot + "/users/root/Namespaces/+accumulo", null));
-    names.add(new PropNode(zkRoot + "/users/root/Namespaces/2", null));
-    names.add(new PropNode(zkRoot + "/users/root/Namespaces/3", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/!0", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/+r", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/1", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/4", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/5", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/6", null));
-    names.add(new PropNode(zkRoot + "/users/root/Tables/7", null));
-    names.add(new PropNode(zkRoot + "/wals/localhost:11000[10000c3202e0003]", null));
-    names.add(new PropNode(
-        zkRoot + "/wals/localhost:11000[10000c3202e0003]/0fef8f3b-d02d-413b-9a27-f1710812b216",
-        null));
-    names.add(new PropNode(
-        zkRoot + "/wals/localhost:11000[10000c3202e0003]/9e410484-e61b-4707-847a-67f96715aa04",
-        null));
-
-    return names;
-  }
-
-  public static class PropNode {
-    private static final byte[] empty = new byte[0];
-    private final String path;
-    private final String value;
-
-    public PropNode(final String path, final String value) {
-      this.path = path;
-      this.value = value;
-    }
-
-    public String getPath() {
-      return path;
-    }
-
-    public byte[] getData() {
-      if (value == null) {
-        return empty;
-      }
-      return value.getBytes(UTF_8);
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java b/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java
deleted file mode 100644
index 647a1e4..0000000
--- a/test/src/main/java/org/apache/accumulo/test/conf/util/TransformTokenIT.java
+++ /dev/null
@@ -1,157 +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.test.conf.util;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
-import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
-import org.apache.accumulo.server.conf.util.TransformToken;
-import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-@Tag(ZOOKEEPER_TESTING_SERVER)
-public class TransformTokenIT {
-
-  @TempDir
-  private static File tempDir;
-
-  private static ZooKeeperTestingServer testZk = null;
-  private static ZooKeeper zooKeeper;
-  private static ZooReaderWriter zrw;
-  private InstanceId instanceId = null;
-
-  private ServerContext context = null;
-  private PropStoreWatcher watcher = null;
-
-  @BeforeAll
-  public static void setupZk() {
-
-    // using default zookeeper port - we don't have a full configuration
-    testZk = new ZooKeeperTestingServer(tempDir);
-    zooKeeper = testZk.getZooKeeper();
-    zrw = testZk.getZooReaderWriter();
-  }
-
-  @AfterAll
-  public static void shutdownZK() throws Exception {
-    testZk.close();
-  }
-
-  @BeforeEach
-  public void testSetup() throws Exception {
-    instanceId = InstanceId.of(UUID.randomUUID());
-
-    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
-    for (LegacyPropData.PropNode node : nodes) {
-      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
-    }
-
-    ZooPropStore propStore = ZooPropStore.initialize(instanceId, zrw);
-
-    context = createMock(ServerContext.class);
-    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
-    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
-    expect(context.getPropStore()).andReturn(propStore).anyTimes();
-
-    watcher = createMock(PropStoreWatcher.class);
-  }
-
-  @AfterEach
-  public void cleanupZnodes() throws Exception {
-    ZooUtil.digestAuth(zooKeeper, ZooKeeperTestingServer.SECRET);
-    ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
-    verify(context, watcher);
-  }
-
-  @Test
-  public void tokenGoPathTest() {
-    replay(context, watcher);
-
-    var sysPropKey = SystemPropKey.of(instanceId);
-
-    TransformToken token = TransformToken.createToken(sysPropKey.getPath(), zrw);
-
-    assertTrue(token.haveTokenOwnership());
-    token.releaseToken();
-    assertFalse(token.haveTokenOwnership());
-
-    // relock by getting a new lock
-    TransformToken lock2 = TransformToken.createToken(sysPropKey.getPath(), zrw);
-    assertTrue(lock2.haveTokenOwnership());
-
-    // fail with a current lock node present
-    TransformToken lock3 = TransformToken.createToken(sysPropKey.getPath(), zrw);
-    assertFalse(lock3.haveTokenOwnership());
-    // and confirm lock still present
-    assertTrue(lock2.haveTokenOwnership());
-  }
-
-  @Test
-  public void failOnInvalidLockTest() throws Exception {
-
-    replay(context, watcher);
-
-    var sysPropKey = SystemPropKey.of(instanceId);
-    var tokenPath = sysPropKey.getPath() + TransformToken.TRANSFORM_TOKEN;
-
-    TransformToken lock = TransformToken.createToken(sysPropKey.getPath(), zrw);
-
-    // force change in lock
-    assertTrue(lock.haveTokenOwnership());
-    zrw.mutateExisting(tokenPath, v -> UUID.randomUUID().toString().getBytes(UTF_8));
-    assertThrows(IllegalStateException.class, lock::releaseToken,
-        "Expected unlock to fail on different UUID");
-
-    // clean-up and get new lock
-    zrw.delete(tokenPath);
-    TransformToken lock3 = TransformToken.createToken(sysPropKey.getPath(), zrw);
-    assertTrue(lock3.haveTokenOwnership());
-    zrw.delete(tokenPath);
-    assertThrows(IllegalStateException.class, lock::releaseToken,
-        "Expected unlock to fail when no lock present");
-
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java b/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java
index 27e26b4..dcd57ec 100644
--- a/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.test.conf.util;
 
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 
 import java.time.Duration;
 
@@ -36,7 +35,6 @@
 import org.slf4j.LoggerFactory;
 
 @Tag(MINI_CLUSTER_ONLY)
-@Tag(SUNNY_DAY)
 public class ZooPropEditorIT extends SharedMiniClusterBase {
 
   private static final Logger LOG = LoggerFactory.getLogger(ZooPropEditorIT.class);
diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java
index 2dde8fa..6c5ac60 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/FateIT.java
@@ -53,8 +53,8 @@
 import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
 import org.apache.accumulo.core.fate.Repo;
 import org.apache.accumulo.core.fate.ZooStore;
+import org.apache.accumulo.core.fate.zookeeper.DistributedReadWriteLock.LockType;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.TraceRepo;
@@ -91,14 +91,15 @@
 
     @Override
     public long isReady(long tid, Manager manager) throws Exception {
-      return Utils.reserveNamespace(manager, namespaceId, tid, false, true, TableOperation.RENAME)
-          + Utils.reserveTable(manager, tableId, tid, true, true, TableOperation.RENAME);
+      return Utils.reserveNamespace(manager, namespaceId, tid, LockType.READ, true,
+          TableOperation.RENAME)
+          + Utils.reserveTable(manager, tableId, tid, LockType.WRITE, true, TableOperation.RENAME);
     }
 
     @Override
     public void undo(long tid, Manager manager) throws Exception {
-      Utils.unreserveNamespace(manager, namespaceId, tid, false);
-      Utils.unreserveTable(manager, tableId, tid, true);
+      Utils.unreserveNamespace(manager, namespaceId, tid, LockType.READ);
+      Utils.unreserveTable(manager, tableId, tid, LockType.WRITE);
     }
 
     @Override
@@ -108,8 +109,8 @@
         FateIT.inCall();
         return null;
       } finally {
-        Utils.unreserveNamespace(manager, namespaceId, tid, false);
-        Utils.unreserveTable(manager, tableId, tid, true);
+        Utils.unreserveNamespace(manager, namespaceId, tid, LockType.READ);
+        Utils.unreserveTable(manager, tableId, tid, LockType.WRITE);
         LOG.debug("Leaving call {}", FateTxId.formatTid(tid));
       }
 
@@ -177,7 +178,7 @@
 
   private static ZooKeeperTestingServer szk = null;
   private static ZooReaderWriter zk = null;
-  private static final String ZK_ROOT = "/accumulo/" + UUID.randomUUID().toString();
+  private static final String ZK_ROOT = "/accumulo/" + UUID.randomUUID();
   private static final NamespaceId NS = NamespaceId.of("testNameSpace");
   private static final TableId TID = TableId.of("testTable");
 
@@ -187,7 +188,7 @@
 
   private enum ExceptionLocation {
     CALL, IS_READY
-  };
+  }
 
   @BeforeAll
   public static void setup() throws Exception {
@@ -209,9 +210,8 @@
   @Timeout(30)
   public void testTransactionStatus() throws Exception {
 
-    final ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store =
-        new AgeOffStore<Manager>(zooStore, 3000, System::currentTimeMillis);
+    final ZooStore<Manager> zooStore = new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk);
+    final AgeOffStore<Manager> store = new AgeOffStore<>(zooStore, 3000, System::currentTimeMillis);
 
     Manager manager = createMock(Manager.class);
     ServerContext sctx = createMock(ServerContext.class);
@@ -220,11 +220,11 @@
     expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes();
     replay(manager, sctx);
 
-    Fate<Manager> fate = new Fate<Manager>(manager, store, TraceRepo::toLogString);
+    ConfigurationCopy config = new ConfigurationCopy();
+    config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
+    config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
+    Fate<Manager> fate = new Fate<>(manager, store, TraceRepo::toLogString, config);
     try {
-      ConfigurationCopy config = new ConfigurationCopy();
-      config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
-      config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
 
       callStarted = new CountDownLatch(1);
       finishCall = new CountDownLatch(1);
@@ -234,9 +234,8 @@
       fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op");
       assertEquals(TStatus.SUBMITTED, getTxStatus(zk, txid));
 
-      fate.startTransactionRunners(config);
       // Wait for the transaction runner to be scheduled.
-      UtilWaitThread.sleep(3000);
+      Thread.sleep(3000);
 
       // wait for call() to be called
       callStarted.await();
@@ -271,9 +270,8 @@
 
   @Test
   public void testCancelWhileNew() throws Exception {
-    final ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store =
-        new AgeOffStore<Manager>(zooStore, 3000, System::currentTimeMillis);
+    final ZooStore<Manager> zooStore = new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk);
+    final AgeOffStore<Manager> store = new AgeOffStore<>(zooStore, 3000, System::currentTimeMillis);
 
     Manager manager = createMock(Manager.class);
     ServerContext sctx = createMock(ServerContext.class);
@@ -282,15 +280,13 @@
     expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes();
     replay(manager, sctx);
 
-    Fate<Manager> fate = new Fate<Manager>(manager, store, TraceRepo::toLogString);
+    ConfigurationCopy config = new ConfigurationCopy();
+    config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
+    config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
+    Fate<Manager> fate = new Fate<>(manager, store, TraceRepo::toLogString, config);
     try {
-      ConfigurationCopy config = new ConfigurationCopy();
-      config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
-      config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
-      fate.startTransactionRunners(config);
-
       // Wait for the transaction runner to be scheduled.
-      UtilWaitThread.sleep(3000);
+      Thread.sleep(3000);
 
       callStarted = new CountDownLatch(1);
       finishCall = new CountDownLatch(1);
@@ -313,10 +309,9 @@
   }
 
   @Test
-  public void testCancelWhileSubmittedNotRunning() throws Exception {
-    final ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store =
-        new AgeOffStore<Manager>(zooStore, 3000, System::currentTimeMillis);
+  public void testCancelWhileSubmittedAndRunning() throws Exception {
+    final ZooStore<Manager> zooStore = new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk);
+    final AgeOffStore<Manager> store = new AgeOffStore<>(zooStore, 3000, System::currentTimeMillis);
 
     Manager manager = createMock(Manager.class);
     ServerContext sctx = createMock(ServerContext.class);
@@ -325,48 +320,14 @@
     expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes();
     replay(manager, sctx);
 
-    Fate<Manager> fate = new Fate<Manager>(manager, store, TraceRepo::toLogString);
     ConfigurationCopy config = new ConfigurationCopy();
     config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
-
-    // Notice that we did not start the transaction runners
-
-    // Wait for the transaction runner to be scheduled.
-    UtilWaitThread.sleep(3000);
-
-    callStarted = new CountDownLatch(1);
-    finishCall = new CountDownLatch(1);
-
-    long txid = fate.startTransaction();
-    LOG.debug("Starting test testCancelWhileSubmitted with {}", FateTxId.formatTid(txid));
-    assertEquals(NEW, getTxStatus(zk, txid));
-    fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op");
-    assertEquals(SUBMITTED, getTxStatus(zk, txid));
-    assertTrue(fate.cancel(txid));
-  }
-
-  @Test
-  public void testCancelWhileSubmittedAndRunning() throws Exception {
-    final ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store =
-        new AgeOffStore<Manager>(zooStore, 3000, System::currentTimeMillis);
-
-    Manager manager = createMock(Manager.class);
-    ServerContext sctx = createMock(ServerContext.class);
-    expect(manager.getContext()).andReturn(sctx).anyTimes();
-    expect(sctx.getZooKeeperRoot()).andReturn(ZK_ROOT).anyTimes();
-    expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes();
-    replay(manager, sctx);
-
-    Fate<Manager> fate = new Fate<Manager>(manager, store, TraceRepo::toLogString);
+    config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
+    Fate<Manager> fate = new Fate<>(manager, store, TraceRepo::toLogString, config);
     try {
-      ConfigurationCopy config = new ConfigurationCopy();
-      config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
-      config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
-      fate.startTransactionRunners(config);
 
       // Wait for the transaction runner to be scheduled.
-      UtilWaitThread.sleep(3000);
+      Thread.sleep(3000);
 
       callStarted = new CountDownLatch(1);
       finishCall = new CountDownLatch(1);
@@ -391,9 +352,8 @@
 
   @Test
   public void testCancelWhileInCall() throws Exception {
-    final ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store =
-        new AgeOffStore<Manager>(zooStore, 3000, System::currentTimeMillis);
+    final ZooStore<Manager> zooStore = new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk);
+    final AgeOffStore<Manager> store = new AgeOffStore<>(zooStore, 3000, System::currentTimeMillis);
 
     Manager manager = createMock(Manager.class);
     ServerContext sctx = createMock(ServerContext.class);
@@ -402,11 +362,14 @@
     expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes();
     replay(manager, sctx);
 
-    Fate<Manager> fate = new Fate<Manager>(manager, store, TraceRepo::toLogString);
+    ConfigurationCopy config = new ConfigurationCopy();
+    config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
+    config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
+    Fate<Manager> fate = new Fate<>(manager, store, TraceRepo::toLogString, config);
     try {
-      ConfigurationCopy config = new ConfigurationCopy();
-      config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
-      config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
+
+      // Wait for the transaction runner to be scheduled.
+      Thread.sleep(3000);
 
       callStarted = new CountDownLatch(1);
       finishCall = new CountDownLatch(1);
@@ -417,10 +380,6 @@
       fate.seedTransaction("TestOperation", txid, new TestOperation(NS, TID), true, "Test Op");
       assertEquals(SUBMITTED, getTxStatus(zk, txid));
 
-      fate.startTransactionRunners(config);
-      // Wait for the transaction runner to be scheduled.
-      UtilWaitThread.sleep(3000);
-
       // wait for call() to be called
       callStarted.await();
       // cancel the transaction
@@ -440,9 +399,8 @@
      * is called and throws an exception (in call() or isReady()). It is then expected that: 1)
      * undo() is called on Repo3, 2) undo() is called on Repo2, 3) undo() is called on Repo1
      */
-    final ZooStore<Manager> zooStore = new ZooStore<Manager>(ZK_ROOT + Constants.ZFATE, zk);
-    final AgeOffStore<Manager> store =
-        new AgeOffStore<Manager>(zooStore, 3000, System::currentTimeMillis);
+    final ZooStore<Manager> zooStore = new ZooStore<>(ZK_ROOT + Constants.ZFATE, zk);
+    final AgeOffStore<Manager> store = new AgeOffStore<>(zooStore, 3000, System::currentTimeMillis);
 
     Manager manager = createMock(Manager.class);
     ServerContext sctx = createMock(ServerContext.class);
@@ -451,15 +409,14 @@
     expect(sctx.getZooReaderWriter()).andReturn(zk).anyTimes();
     replay(manager, sctx);
 
-    Fate<Manager> fate = new Fate<Manager>(manager, store, TraceRepo::toLogString);
+    ConfigurationCopy config = new ConfigurationCopy();
+    config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
+    config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
+    Fate<Manager> fate = new Fate<>(manager, store, TraceRepo::toLogString, config);
     try {
-      ConfigurationCopy config = new ConfigurationCopy();
-      config.set(Property.GENERAL_THREADPOOL_SIZE, "2");
-      config.set(Property.MANAGER_FATE_THREADPOOL_SIZE, "1");
-      fate.startTransactionRunners(config);
 
       // Wait for the transaction runner to be scheduled.
-      UtilWaitThread.sleep(3000);
+      Thread.sleep(3000);
 
       List<String> expectedUndoOrder = List.of("OP3", "OP2", "OP1");
       /*
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
index d5fb850..694f4ee 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java
@@ -72,42 +72,6 @@
     assertTrue(e.getMessage().toLowerCase().contains("closed"));
   }
 
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testGetConnectorFromAccumuloClient() throws Exception {
-    AccumuloClient client = Accumulo.newClient().from(getClientProps()).build();
-    org.apache.accumulo.core.client.Connector c =
-        org.apache.accumulo.core.client.Connector.from(client);
-    assertEquals(client.whoami(), c.whoami());
-
-    // this should cause the connector to stop functioning
-    client.close();
-
-    expectClosed(c::tableOperations);
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testGetAccumuloClientFromConnector() throws Exception {
-    try (AccumuloClient client1 = Accumulo.newClient().from(getClientProps()).build()) {
-      org.apache.accumulo.core.client.Connector c =
-          org.apache.accumulo.core.client.Connector.from(client1);
-
-      String tableName = getUniqueNames(1)[0];
-
-      c.tableOperations().create(tableName);
-
-      try (AccumuloClient client2 = org.apache.accumulo.core.client.Connector.newClient(c)) {
-        assertTrue(client2.tableOperations().list().contains(tableName));
-      }
-
-      // closing client2 should not have had an impact on the connector or client1
-
-      assertTrue(client1.tableOperations().list().contains(tableName));
-      assertTrue(c.tableOperations().list().contains(tableName));
-    }
-  }
-
   @Test
   public void testAccumuloClientBuilder() throws Exception {
     AccumuloClient c = Accumulo.newClient().from(getClientProps()).build();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
index 437ea7b..76c62a5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AddSplitIT.java
@@ -18,14 +18,11 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
 import java.time.Duration;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map.Entry;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -61,7 +58,7 @@
 
       c.tableOperations().addSplits(tableName, splits);
 
-      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
 
       Collection<Text> actualSplits = c.tableOperations().listSplits(tableName);
 
@@ -81,7 +78,7 @@
 
       c.tableOperations().addSplits(tableName, splits);
 
-      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
 
       actualSplits = c.tableOperations().listSplits(tableName);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java
index a80717c..376dad3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -32,17 +33,21 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.manager.state.MetaDataTableScanner;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 
 public class AssignLocationModeIT extends ConfigurableMacBase {
 
+  @SuppressWarnings("deprecation")
+  private static final Property DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY =
+      Property.TSERV_LAST_LOCATION_MODE;
+
   @Override
   protected Duration defaultTimeout() {
     return Duration.ofMinutes(2);
@@ -50,7 +55,7 @@
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration fsConf) {
-    cfg.setProperty(Property.TSERV_LAST_LOCATION_MODE, "assignment");
+    cfg.setProperty(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY, "assignment");
   }
 
   @Test
@@ -61,11 +66,9 @@
       c.tableOperations().create(tableName);
       String tableId = c.tableOperations().tableIdMap().get(tableName);
       // wait for the table to be online
-      TabletLocationState newTablet;
-      do {
-        UtilWaitThread.sleep(250);
-        newTablet = getTabletLocationState(c, tableId);
-      } while (newTablet.current == null);
+      Wait.waitFor(() -> getTabletLocationState(c, tableId) != null, SECONDS.toMillis(60), 250);
+
+      TabletLocationState newTablet = getTabletLocationState(c, tableId);
       // this would be null if the mode was not "assign"
       assertEquals(newTablet.getCurrentServer(), newTablet.getLastServer());
       assertNull(newTablet.future);
@@ -78,7 +81,7 @@
       }
       // assert that the default mode is "assign"
       assertEquals("assignment", c.instanceOperations().getSystemConfiguration()
-          .get(Property.TSERV_LAST_LOCATION_MODE.getKey()));
+          .get(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY.getKey()));
 
       // last location should not be set yet
       TabletLocationState unflushed = getTabletLocationState(c, tableId);
@@ -104,7 +107,8 @@
 
   private TabletLocationState getTabletLocationState(AccumuloClient c, String tableId) {
     try (MetaDataTableScanner s = new MetaDataTableScanner((ClientContext) c,
-        new Range(TabletsSection.encodeRow(TableId.of(tableId), null)), MetadataTable.NAME)) {
+        new Range(TabletsSection.encodeRow(TableId.of(tableId), null)),
+        AccumuloTable.METADATA.tableName())) {
       return s.next();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java
index 0bb4f81..fd9a908 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java
@@ -24,11 +24,11 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.manager.Manager;
+import org.apache.accumulo.test.util.Wait;
 import org.junit.jupiter.api.Test;
 
 public class BackupManagerIT extends ConfigurableMacBase {
@@ -41,27 +41,28 @@
   @Test
   public void test() throws Exception {
     // wait for manager
-    UtilWaitThread.sleep(1000);
+    Thread.sleep(1000);
     // create a backup
     Process backup = exec(Manager.class);
     try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
       ZooReaderWriter writer = getCluster().getServerContext().getZooReaderWriter();
       String root = "/accumulo/" + client.instanceOperations().getInstanceId();
-      List<String> children;
+
       // wait for 2 lock entries
-      do {
-        UtilWaitThread.sleep(100);
-        var path = ServiceLock.path(root + Constants.ZMANAGER_LOCK);
-        children = ServiceLock.validateAndSort(path, writer.getChildren(path.toString()));
-      } while (children.size() != 2);
+      var path = ServiceLock.path(root + Constants.ZMANAGER_LOCK);
+      Wait.waitFor(
+          () -> ServiceLock.validateAndSort(path, writer.getChildren(path.toString())).size() == 2);
+
       // wait for the backup manager to learn to be the backup
-      UtilWaitThread.sleep(1000);
+      Thread.sleep(1000);
       // generate a false zookeeper event
+      List<String> children =
+          ServiceLock.validateAndSort(path, writer.getChildren(path.toString()));
       String lockPath = root + Constants.ZMANAGER_LOCK + "/" + children.get(0);
       byte[] data = writer.getData(lockPath);
       writer.getZooKeeper().setData(lockPath, data, -1);
       // let it propagate
-      UtilWaitThread.sleep(500);
+      Thread.sleep(500);
       // kill the manager by removing its lock
       writer.recursiveDelete(lockPath, NodeMissingPolicy.FAIL);
       // ensure the backup becomes the manager
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
index 417459a..75ee531 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java
@@ -18,12 +18,11 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
 import java.util.EnumSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -61,7 +60,7 @@
       }
 
       c.tableOperations().flush(tableName, null, null, false);
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
 
       // minc should fail, so there should be no files
       FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 0, 0);
@@ -75,7 +74,7 @@
         c.tableOperations().removeIterator(tableName, BadIterator.class.getSimpleName(),
             EnumSet.of(IteratorScope.minc));
 
-        sleepUninterruptibly(5, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(5));
 
         // minc should complete
         FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 1, 1);
@@ -95,12 +94,12 @@
         }
 
         // make sure property is given time to propagate
-        sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+        Thread.sleep(500);
 
         c.tableOperations().flush(tableName, null, null, false);
 
         // make sure the flush has time to start
-        sleepUninterruptibly(1, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(1));
 
         // this should not hang
         c.tableOperations().delete(tableName);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
index b63e421..0d5ff21 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
@@ -34,12 +34,11 @@
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.miniclusterImpl.ProcessReference;
@@ -84,7 +83,7 @@
         assertEquals(0, Runtime.getRuntime()
             .exec(new String[] {"kill", "-SIGSTOP", Integer.toString(pid)}).waitFor());
       }
-      UtilWaitThread.sleep(20_000);
+      Thread.sleep(20_000);
       for (int pid : tserverPids) {
         assertEquals(0, Runtime.getRuntime()
             .exec(new String[] {"kill", "-SIGCONT", Integer.toString(pid)}).waitFor());
@@ -96,7 +95,7 @@
       c.tableOperations().addSplits("test", splits);
       // Ensure all of the tablets are actually assigned
       assertEquals(0, Iterables.size(c.createScanner("test", Authorizations.EMPTY)));
-      UtilWaitThread.sleep(30_000);
+      Thread.sleep(30_000);
       checkBalance(c);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
index 72889e0..411258c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -25,7 +25,6 @@
 import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -37,15 +36,15 @@
 import org.apache.accumulo.core.clientImpl.Credentials;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.commons.lang3.math.NumberUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -86,13 +85,10 @@
   public void setupTables() throws AccumuloException, AccumuloSecurityException,
       TableExistsException, TableNotFoundException {
     accumuloClient = Accumulo.newClient().from(getClientProps()).build();
+
     // Need at least two tservers -- wait for them to start before failing
-    for (int retries = 0; retries < 5; ++retries) {
-      if (accumuloClient.instanceOperations().getTabletServers().size() >= 2) {
-        break;
-      }
-      UtilWaitThread.sleep(TimeUnit.SECONDS.toMillis(2));
-    }
+    Wait.waitFor(() -> accumuloClient.instanceOperations().getTabletServers().size() >= 2);
+
     assumeTrue(accumuloClient.instanceOperations().getTabletServers().size() >= 2,
         "Not enough tservers to run test");
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
index 5eb35f3..19b7eed 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchScanSplitIT.java
@@ -18,14 +18,13 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -79,7 +78,7 @@
 
       Collection<Text> splits = c.tableOperations().listSplits(tableName);
       while (splits.size() < 2) {
-        sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+        Thread.sleep(1);
         splits = c.tableOperations().listSplits(tableName);
       }
 
@@ -88,7 +87,7 @@
       HashMap<Text,Value> expected = new HashMap<>();
       ArrayList<Range> ranges = new ArrayList<>();
       for (int i = 0; i < 100; i++) {
-        int r = random.nextInt(numRows);
+        int r = RANDOM.get().nextInt(numRows);
         Text row = new Text(String.format("%09x", r));
         expected.put(row, new Value(String.format("%016x", numRows - r)));
         ranges.add(new Range(row));
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
index 101c9fc..7aa6886 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BatchWriterFlushIT.java
@@ -18,7 +18,10 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+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.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
@@ -34,7 +37,6 @@
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -81,14 +83,14 @@
     // should automatically flush after 2 seconds
     try (
         BatchWriter bw = client.createBatchWriter(tableName,
-            new BatchWriterConfig().setMaxLatency(1000, TimeUnit.MILLISECONDS));
+            new BatchWriterConfig().setMaxLatency(1000, MILLISECONDS));
         Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
 
       Mutation m = new Mutation(new Text(String.format("r_%10d", 1)));
       m.put("cf", "cq", "1");
       bw.addMutation(m);
 
-      sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+      Thread.sleep(500);
 
       int count = Iterators.size(scanner.iterator());
 
@@ -96,7 +98,7 @@
         throw new Exception("Flushed too soon");
       }
 
-      sleepUninterruptibly(1500, TimeUnit.MILLISECONDS);
+      Thread.sleep(1500);
 
       count = Iterators.size(scanner.iterator());
 
@@ -124,7 +126,7 @@
         // do a few random lookups into the data just flushed
 
         for (int k = 0; k < 10; k++) {
-          int rowToLookup = random.nextInt(NUM_TO_FLUSH) + i * NUM_TO_FLUSH;
+          int rowToLookup = RANDOM.get().nextInt(NUM_TO_FLUSH) + i * NUM_TO_FLUSH;
 
           scanner.setRange(new Range(new Text(String.format("r_%10d", rowToLookup))));
 
@@ -218,7 +220,7 @@
       threads.prestartAllCoreThreads();
 
       BatchWriterConfig cfg = new BatchWriterConfig();
-      cfg.setMaxLatency(10, TimeUnit.SECONDS);
+      cfg.setMaxLatency(10, SECONDS);
       cfg.setMaxMemory(1 * 1024 * 1024);
       cfg.setMaxWriteThreads(NUM_THREADS);
       final BatchWriter bw = c.createBatchWriter(tableName, cfg);
@@ -238,7 +240,7 @@
         });
       }
       threads.shutdown();
-      threads.awaitTermination(3, TimeUnit.MINUTES);
+      threads.awaitTermination(3, MINUTES);
       bw.close();
       try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
         for (Entry<Key,Value> e : scanner) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
index dc68fb9..5844141 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java
@@ -26,8 +26,7 @@
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.Property;
-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.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -54,18 +53,18 @@
   @Test
   public void test() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().addSplits(MetadataTable.NAME,
+      c.tableOperations().addSplits(AccumuloTable.METADATA.tableName(),
           FunctionalTestUtils.splits("0 1 2 3 4 5 6 7 8 9 a".split(" ")));
       String[] names = getUniqueNames(10);
       for (String name : names) {
         c.tableOperations().create(name);
-        c.tableOperations().flush(MetadataTable.NAME, null, null, true);
-        c.tableOperations().flush(RootTable.NAME, null, null, true);
+        c.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+        c.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
       }
       cluster.stop();
       cluster.start();
-      assertTrue(
-          c.createScanner(RootTable.NAME, Authorizations.EMPTY).stream().findAny().isPresent());
+      assertTrue(c.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY).stream()
+          .findAny().isPresent());
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java
index 888a088..2450f70 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BinaryStressIT.java
@@ -34,7 +34,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
@@ -102,7 +102,7 @@
       BinaryIT.runTest(c, tableName);
       String id = c.tableOperations().tableIdMap().get(tableName);
       Set<Text> tablets = new HashSet<>();
-      try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.setRange(Range.prefix(id));
         for (Entry<Key,Value> entry : s) {
           tablets.add(entry.getKey().getRow());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
index 0ccf7b3..8aed4dc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BloomFilterIT.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -42,7 +44,6 @@
 import org.apache.accumulo.core.file.keyfunctor.ColumnQualifierFunctor;
 import org.apache.accumulo.core.file.keyfunctor.RowFunctor;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -136,8 +137,8 @@
         c.tableOperations().setProperty(tables[3], Property.TABLE_BLOOM_KEY_FUNCTOR.getKey(),
             RowFunctor.class.getName());
 
-        // ensure the updates to zookeeper propagate
-        UtilWaitThread.sleep(500);
+        // allow the updates to zookeeper propagate
+        Thread.sleep(500);
 
         c.tableOperations().compact(tables[3], null, null, false, true);
         c.tableOperations().compact(tables[0], null, null, false, true);
@@ -187,7 +188,7 @@
     Text row = new Text("row"), cq = new Text("cq"), cf = new Text("cf");
 
     for (int i = 0; i < num; ++i) {
-      Long k = ((random.nextLong() & 0x7fffffffffffffffL) % (end - start)) + start;
+      Long k = ((RANDOM.get().nextLong() & 0x7fffffffffffffffL) % (end - start)) + start;
       key.set(String.format("k_%010d", k));
       Range range = null;
       Key acuKey;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
index 98f3e47..cb4b62d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java
@@ -18,19 +18,25 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.Constants;
@@ -47,39 +53,47 @@
 import org.apache.accumulo.core.client.rfile.RFileWriter;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
+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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 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.TablePermission;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 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.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.manager.tableOps.bulkVer1.BulkImport;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.tablets.UniqueNameAllocator;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher.ZooArbitrator;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TServiceClient;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.zookeeper.KeeperException;
 import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.net.HostAndPort;
 
 public class BulkFailureIT extends AccumuloClusterHarness {
 
+  private static final Logger LOG = LoggerFactory.getLogger(BulkFailureIT.class);
+
   interface Loader {
     void load(long txid, ClientContext context, KeyExtent extent, Path path, long size,
         boolean expectFailure) throws Exception;
@@ -89,13 +103,104 @@
   public void testImportCompactionImport() throws Exception {
     String[] tables = getUniqueNames(2);
 
-    // run test calling old bulk import RPCs
-    runTest(tables[0], 99999999L, BulkFailureIT::oldLoad);
-
     // run test calling new bulk import RPCs
     runTest(tables[1], 22222222L, BulkFailureIT::newLoad);
   }
 
+  private static Path createNewBulkDir(ServerContext context, VolumeManager fs, String sourceDir,
+      TableId tableId) throws IOException {
+    Path tableDir = fs.matchingFileSystem(new Path(sourceDir), context.getTablesDirs());
+    if (tableDir == null) {
+      throw new IOException(
+          sourceDir + " is not in the same file system as any volume configured for Accumulo");
+    }
+
+    Path directory = new Path(tableDir, tableId.canonical());
+    fs.mkdirs(directory);
+
+    // only one should be able to create the lock file
+    // the purpose of the lock file is to avoid a race
+    // condition between the call to fs.exists() and
+    // fs.mkdirs()... if only hadoop had a mkdir() function
+    // that failed when the dir existed
+
+    UniqueNameAllocator namer = context.getUniqueNameAllocator();
+
+    while (true) {
+      Path newBulkDir = new Path(directory, Constants.BULK_PREFIX + namer.getNextName());
+      if (fs.exists(newBulkDir)) { // sanity check
+        throw new IOException("Dir exist when it should not " + newBulkDir);
+      }
+      if (fs.mkdirs(newBulkDir)) {
+        return newBulkDir;
+      }
+
+      sleepUninterruptibly(3, TimeUnit.SECONDS);
+    }
+  }
+
+  public static String prepareBulkImport(ServerContext manager, final VolumeManager fs, String dir,
+      TableId tableId, long tid) throws Exception {
+    final Path bulkDir = createNewBulkDir(manager, fs, dir, tableId);
+
+    manager.getAmple().addBulkLoadInProgressFlag(
+        "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName(), tid);
+
+    Path dirPath = new Path(dir);
+    FileStatus[] dataFiles = fs.listStatus(dirPath);
+
+    final UniqueNameAllocator namer = manager.getUniqueNameAllocator();
+
+    AccumuloConfiguration serverConfig = manager.getConfiguration();
+    int numThreads = serverConfig.getCount(Property.MANAGER_RENAME_THREADS);
+    ExecutorService workers = ThreadPools.getServerThreadPools().getPoolBuilder("bulk rename")
+        .numCoreThreads(numThreads).build();
+    List<Future<Exception>> results = new ArrayList<>();
+
+    for (FileStatus file : dataFiles) {
+      final FileStatus fileStatus = file;
+      results.add(workers.submit(() -> {
+        try {
+          String[] sa = fileStatus.getPath().getName().split("\\.");
+          String extension = "";
+          if (sa.length > 1) {
+            extension = sa[sa.length - 1];
+
+            if (!FileOperations.getValidExtensions().contains(extension)) {
+              LOG.warn("{} does not have a valid extension, ignoring", fileStatus.getPath());
+              return null;
+            }
+          } else {
+            LOG.warn("{} does not have any extension, ignoring", fileStatus.getPath());
+            return null;
+          }
+
+          String newName = "I" + namer.getNextName() + "." + extension;
+          Path newPath = new Path(bulkDir, newName);
+          try {
+            fs.rename(fileStatus.getPath(), newPath);
+            LOG.debug("Moved {} to {}", fileStatus.getPath(), newPath);
+          } catch (IOException E1) {
+            LOG.error("Could not move: {} {}", fileStatus.getPath(), E1.getMessage());
+          }
+
+        } catch (Exception ex) {
+          return ex;
+        }
+        return null;
+      }));
+    }
+    workers.shutdown();
+    while (!workers.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {}
+
+    for (Future<Exception> ex : results) {
+      if (ex.get() != null) {
+        throw ex.get();
+      }
+    }
+    return bulkDir.toString();
+  }
+
   /**
    * This test verifies two things. First it ensures that after a bulk imported file is compacted
    * that import request are ignored. Second it ensures that after the bulk import transaction is
@@ -125,8 +230,8 @@
       VolumeManager vm = asCtx.getVolumeManager();
 
       // move the file into a directory for the table and rename the file to something unique
-      String bulkDir =
-          BulkImport.prepareBulkImport(asCtx, vm, testFile, TableId.of(tableId), fateTxid);
+      String bulkDir = prepareBulkImport(asCtx, vm, testFile, TableId.of(tableId), fateTxid);
+      assertNotNull(bulkDir);
 
       // determine the files new name and path
       FileStatus status = fs.listStatus(new Path(bulkDir))[0];
@@ -170,10 +275,11 @@
       // After this, all load request should fail.
       ZooArbitrator.stop(asCtx, Constants.BULK_ARBITRATOR_TYPE, fateTxid);
 
-      c.securityOperations().grantTablePermission(c.whoami(), MetadataTable.NAME,
+      c.securityOperations().grantTablePermission(c.whoami(), AccumuloTable.METADATA.tableName(),
           TablePermission.WRITE);
 
-      BatchDeleter bd = c.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 1);
+      BatchDeleter bd =
+          c.createBatchDeleter(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY, 1);
       bd.setRanges(Collections.singleton(extent.toMetaRange()));
       bd.fetchColumnFamily(BulkFileColumnFamily.NAME);
       bd.delete();
@@ -239,48 +345,26 @@
       throws TableNotFoundException {
     HashSet<Path> files = new HashSet<>();
 
-    Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    Scanner scanner =
+        connector.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY);
     scanner.setRange(extent.toMetaRange());
     scanner.fetchColumnFamily(fam);
 
     for (Entry<Key,Value> entry : scanner) {
-      files.add(new Path(entry.getKey().getColumnQualifierData().toString()));
+      files.add(StoredTabletFile.of(entry.getKey().getColumnQualifier()).getPath());
     }
 
     return files;
   }
 
-  private static void oldLoad(long txid, ClientContext context, KeyExtent extent, Path path,
-      long size, boolean expectFailure) throws Exception {
-
-    TabletClientService.Iface client = getClient(context, extent);
-    try {
-
-      Map<String,MapFileInfo> val = Map.of(path.toString(), new MapFileInfo(size));
-      Map<KeyExtent,Map<String,MapFileInfo>> files = Map.of(extent, val);
-
-      client.bulkImport(TraceUtil.traceInfo(), context.rpcCreds(), txid, files.entrySet().stream()
-          .collect(Collectors.toMap(entry -> entry.getKey().toThrift(), Entry::getValue)), false);
-      if (expectFailure) {
-        fail("Expected RPC to fail");
-      }
-    } catch (TApplicationException tae) {
-      if (!expectFailure) {
-        throw tae;
-      }
-    } finally {
-      ThriftUtil.returnClient((TServiceClient) client, context);
-    }
-  }
-
   private static void newLoad(long txid, ClientContext context, KeyExtent extent, Path path,
       long size, boolean expectFailure) throws Exception {
 
-    TabletClientService.Iface client = getClient(context, extent);
+    TabletIngestClientService.Iface client = getClient(context, extent);
     try {
 
-      Map<String,MapFileInfo> val = Map.of(path.getName(), new MapFileInfo(size));
-      Map<KeyExtent,Map<String,MapFileInfo>> files = Map.of(extent, val);
+      Map<String,DataFileInfo> val = Map.of(path.getName(), new DataFileInfo(size));
+      Map<KeyExtent,Map<String,DataFileInfo>> files = Map.of(extent, val);
 
       client.loadFiles(TraceUtil.traceInfo(), context.rpcCreds(), txid, path.getParent().toString(),
           files.entrySet().stream().collect(
@@ -298,19 +382,19 @@
     }
   }
 
-  protected static TabletClientService.Iface getClient(ClientContext context, KeyExtent extent)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
+  protected static TabletIngestClientService.Iface getClient(ClientContext context,
+      KeyExtent extent) throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
       TTransportException {
     TabletLocator locator = TabletLocator.getLocator(context, extent.tableId());
 
     locator.invalidateCache(extent);
 
     HostAndPort location = HostAndPort
-        .fromString(locator.locateTablet(context, new Text(""), false, true).tablet_location);
+        .fromString(locator.locateTablet(context, new Text(""), false, true).getTserverLocation());
 
-    long timeInMillis = context.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
-    TabletClientService.Iface client =
-        ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeInMillis);
+    long timeInMillis = context.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
+    TabletIngestClientService.Iface client =
+        ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeInMillis);
     return client;
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
index ae67f35..1f4e250 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkIT.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
-
 import java.io.IOException;
 import java.time.Duration;
 
@@ -35,13 +33,11 @@
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 
 /**
- * Tests Old and New Bulk import
+ * Tests Bulk import
  */
-@Tag(SUNNY_DAY)
 public class BulkIT extends AccumuloClusterHarness {
 
   private static final int N = 100000;
@@ -56,20 +52,12 @@
   public void test() throws Exception {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       runTest(client, getCluster().getFileSystem(), getCluster().getTemporaryPath(),
-          getUniqueNames(1)[0], this.getClass().getName(), testName(), false);
-    }
-  }
-
-  @Test
-  public void testOld() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      runTest(client, getCluster().getFileSystem(), getCluster().getTemporaryPath(),
-          getUniqueNames(1)[0], this.getClass().getName(), testName(), true);
+          getUniqueNames(1)[0], this.getClass().getName(), testName());
     }
   }
 
   static void runTest(AccumuloClient c, FileSystem fs, Path basePath, String tableName,
-      String filePrefix, String dirSuffix, boolean useOld) throws Exception {
+      String filePrefix, String dirSuffix) throws Exception {
     c.tableOperations().create(tableName);
     Path base = new Path(basePath, "testBulkFail_" + dirSuffix);
     fs.delete(base, true);
@@ -98,7 +86,7 @@
     // create an rfile with one entry, there was a bug with this:
     TestIngest.ingest(c, fs, params);
 
-    bulkLoad(c, tableName, bulkFailures, files, useOld);
+    bulkLoad(c, tableName, bulkFailures, files);
     VerifyParams verifyParams = new VerifyParams(c.properties(), tableName, N);
     verifyParams.random = 56;
     for (int i = 0; i < COUNT; i++) {
@@ -110,34 +98,26 @@
     VerifyIngest.verifyIngest(c, verifyParams);
   }
 
-  @SuppressWarnings("deprecation")
-  private static void bulkLoad(AccumuloClient c, String tableName, Path bulkFailures, Path files,
-      boolean useOld)
+  private static void bulkLoad(AccumuloClient c, String tableName, Path bulkFailures, Path files)
       throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException {
     // Make sure the server can modify the files
-    if (useOld) {
-      c.tableOperations().importDirectory(tableName, files.toString(), bulkFailures.toString(),
-          false);
-    } else {
-      // not appending the 'ignoreEmptyDir' method defaults to not ignoring empty directories.
+    // not appending the 'ignoreEmptyDir' method defaults to not ignoring empty directories.
+    c.tableOperations().importDirectory(files.toString()).to(tableName).load();
+    try {
+      // if run again, the expected IllegalArgrumentException is thrown
       c.tableOperations().importDirectory(files.toString()).to(tableName).load();
-      try {
-        // if run again, the expected IllegalArgrumentException is thrown
-        c.tableOperations().importDirectory(files.toString()).to(tableName).load();
-      } catch (IllegalArgumentException ex) {
-        // expected exception to be thrown
-      }
-      // re-run using the ignoreEmptyDir option and no error should be thrown since empty
-      // directories will be ignored
-      c.tableOperations().importDirectory(files.toString()).to(tableName).ignoreEmptyDir(true)
+    } catch (IllegalArgumentException ex) {
+      // expected exception to be thrown
+    }
+    // re-run using the ignoreEmptyDir option and no error should be thrown since empty
+    // directories will be ignored
+    c.tableOperations().importDirectory(files.toString()).to(tableName).ignoreEmptyDir(true).load();
+    try {
+      // setting ignoreEmptyDir to false, explicitly, results in exception being thrown again.
+      c.tableOperations().importDirectory(files.toString()).to(tableName).ignoreEmptyDir(false)
           .load();
-      try {
-        // setting ignoreEmptyDir to false, explicitly, results in exception being thrown again.
-        c.tableOperations().importDirectory(files.toString()).to(tableName).ignoreEmptyDir(false)
-            .load();
-      } catch (IllegalArgumentException ex) {
-        // expected exception to be thrown
-      }
+    } catch (IllegalArgumentException ex) {
+      // expected exception to be thrown
     }
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java
index a8aebd2..06fdb69 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java
@@ -71,8 +71,9 @@
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.file.rfile.RFile;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
@@ -103,11 +104,6 @@
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
-/**
- * Tests new bulk import technique. For the old technique see {@link BulkOldIT}
- *
- * @since 2.0
- */
 public class BulkNewIT extends SharedMiniClusterBase {
 
   @Override
@@ -616,7 +612,7 @@
       for (TabletMetadata tablet : tablets) {
         assertTrue(tablet.getLoaded().isEmpty());
 
-        Set<String> fileHashes = tablet.getFiles().stream().map(f -> hash(f.getMetaUpdateDelete()))
+        Set<String> fileHashes = tablet.getFiles().stream().map(f -> hash(f.getMetadataPath()))
             .collect(Collectors.toSet());
 
         String endRow = tablet.getEndRow() == null ? "null" : tablet.getEndRow().toString();
@@ -651,7 +647,8 @@
     FileSystem fs = getCluster().getFileSystem();
     String filename = file + RFile.EXTENSION;
     try (FileSKVWriter writer = FileOperations.getInstance().newWriterBuilder()
-        .forFile(filename, fs, fs.getConf(), NoCryptoServiceFactory.NONE)
+        .forFile(UnreferencedTabletFile.of(fs, new Path(filename)), fs, fs.getConf(),
+            NoCryptoServiceFactory.NONE)
         .withTableConfiguration(aconf).build()) {
       writer.startDefaultLocalityGroup();
       for (int i = s; i <= e; i++) {
@@ -705,12 +702,13 @@
   static void setupBulkConstraint(String principal, AccumuloClient c)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     // add a constraint to the metadata table that disallows bulk import files to be added
-    c.securityOperations().grantTablePermission(principal, MetadataTable.NAME,
+    c.securityOperations().grantTablePermission(principal, AccumuloTable.METADATA.tableName(),
         TablePermission.WRITE);
-    c.securityOperations().grantTablePermission(principal, MetadataTable.NAME,
+    c.securityOperations().grantTablePermission(principal, AccumuloTable.METADATA.tableName(),
         TablePermission.ALTER_TABLE);
 
-    c.tableOperations().addConstraint(MetadataTable.NAME, NoBulkConstratint.class.getName());
+    c.tableOperations().addConstraint(AccumuloTable.METADATA.tableName(),
+        NoBulkConstratint.class.getName());
 
     var metaConstraints = new MetadataConstraints();
     SystemEnvironment env = EasyMock.createMock(SystemEnvironment.class);
@@ -720,7 +718,7 @@
 
     // wait for the constraint to be active on the metadata table
     Wait.waitFor(() -> {
-      try (var bw = c.createBatchWriter(MetadataTable.NAME)) {
+      try (var bw = c.createBatchWriter(AccumuloTable.METADATA.tableName())) {
         Mutation m = new Mutation("~garbage");
         m.put("", "", NoBulkConstratint.CANARY_VALUE);
         // This test assume the metadata constraint check will not flag this mutation, the following
@@ -735,7 +733,7 @@
     });
 
     // delete the junk added to the metadata table
-    try (var bw = c.createBatchWriter(MetadataTable.NAME)) {
+    try (var bw = c.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       Mutation m = new Mutation("~garbage");
       m.putDelete("", "");
       bw.addMutation(m);
@@ -744,12 +742,12 @@
 
   static void removeBulkConstraint(String principal, AccumuloClient c)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    int constraintNum = c.tableOperations().listConstraints(MetadataTable.NAME)
+    int constraintNum = c.tableOperations().listConstraints(AccumuloTable.METADATA.tableName())
         .get(NoBulkConstratint.class.getName());
-    c.tableOperations().removeConstraint(MetadataTable.NAME, constraintNum);
-    c.securityOperations().revokeTablePermission(principal, MetadataTable.NAME,
+    c.tableOperations().removeConstraint(AccumuloTable.METADATA.tableName(), constraintNum);
+    c.securityOperations().revokeTablePermission(principal, AccumuloTable.METADATA.tableName(),
         TablePermission.WRITE);
-    c.securityOperations().revokeTablePermission(principal, MetadataTable.NAME,
+    c.securityOperations().revokeTablePermission(principal, AccumuloTable.METADATA.tableName(),
         TablePermission.ALTER_TABLE);
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkOldIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkOldIT.java
deleted file mode 100644
index d8df93b..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkOldIT.java
+++ /dev/null
@@ -1,202 +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.test.functional;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.io.IOException;
-import java.time.Duration;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-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.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.rfile.RFile;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.minicluster.MemoryUnit;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-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.junit.jupiter.api.Test;
-
-/**
- * Tests old bulk import technique. For new bulk import see {@link BulkNewIT}
- */
-public class BulkOldIT extends AccumuloClusterHarness {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(4);
-  }
-
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration conf) {
-    cfg.setMemory(ServerType.TABLET_SERVER, 512, MemoryUnit.MEGABYTE);
-    // lowering this because the test testExceptionInMetadataUpdate() will cause retries and the
-    // default takes forever
-    cfg.setProperty(Property.TSERV_BULK_RETRY, "2");
-  }
-
-  // suppress importDirectory deprecated since this is the only test for legacy technique
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testBulkFile() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-      SortedSet<Text> splits = new TreeSet<>();
-      for (String split : "0333 0666 0999 1333 1666".split(" ")) {
-        splits.add(new Text(split));
-      }
-      NewTableConfiguration ntc = new NewTableConfiguration().withSplits(splits);
-      c.tableOperations().create(tableName, ntc);
-      Configuration conf = new Configuration();
-      AccumuloConfiguration aconf = getCluster().getServerContext().getConfiguration();
-      FileSystem fs = getCluster().getFileSystem();
-      String rootPath = cluster.getTemporaryPath().toString();
-
-      String dir = rootPath + "/bulk_test_diff_files_89723987592_" + getUniqueNames(1)[0];
-
-      fs.delete(new Path(dir), true);
-
-      writeData(conf, aconf, fs, dir, "f1", 0, 333);
-      writeData(conf, aconf, fs, dir, "f2", 334, 999);
-      writeData(conf, aconf, fs, dir, "f3", 1000, 1999);
-
-      String failDir = dir + "_failures";
-      Path failPath = new Path(failDir);
-      fs.delete(failPath, true);
-      fs.mkdirs(failPath);
-      fs.deleteOnExit(failPath);
-
-      // Ensure server can read/modify files
-      c.tableOperations().importDirectory(tableName, dir, failDir, false);
-
-      if (fs.listStatus(failPath).length > 0) {
-        throw new Exception("Some files failed to bulk import");
-      }
-
-      FunctionalTestUtils.checkRFiles(c, tableName, 6, 6, 1, 1);
-
-      verifyData(c, tableName, 0, 1999);
-    }
-
-  }
-
-  // test case where the metadata data update throws an exception
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testExceptionInMetadataUpdate() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      // after setting this up, bulk imports should fail
-      BulkNewIT.setupBulkConstraint(getAdminPrincipal(), c);
-
-      String tableName = getUniqueNames(1)[0];
-
-      c.tableOperations().create(tableName);
-      Configuration conf = new Configuration();
-      AccumuloConfiguration aconf = getCluster().getServerContext().getConfiguration();
-      FileSystem fs = getCluster().getFileSystem();
-      String rootPath = cluster.getTemporaryPath().toString();
-
-      String dir = rootPath + "/bulk_test_diff_files_89723987592_" + getUniqueNames(1)[0];
-
-      fs.delete(new Path(dir), true);
-
-      writeData(conf, aconf, fs, dir, "f1", 0, 333);
-
-      String failDir = dir + "_failures";
-      Path failPath = new Path(failDir);
-      fs.delete(failPath, true);
-      fs.mkdirs(failPath);
-      fs.deleteOnExit(failPath);
-
-      // this should fail and it should copy the file to the fail dir
-      c.tableOperations().importDirectory(tableName, dir, failDir, false);
-
-      if (fs.listStatus(failPath).length < 1) {
-        throw new Exception("Expected files in failure directory");
-      }
-
-      try (var scanner = c.createScanner(tableName)) {
-        // verify the table is empty
-        assertEquals(0, scanner.stream().count());
-      }
-
-      BulkNewIT.removeBulkConstraint(getAdminPrincipal(), c);
-    }
-  }
-
-  private void writeData(Configuration conf, AccumuloConfiguration aconf, FileSystem fs, String dir,
-      String file, int start, int end) throws IOException, Exception {
-    FileSKVWriter writer1 = FileOperations.getInstance().newWriterBuilder()
-        .forFile(dir + "/" + file + "." + RFile.EXTENSION, fs, conf, NoCryptoServiceFactory.NONE)
-        .withTableConfiguration(aconf).build();
-    writer1.startDefaultLocalityGroup();
-    for (int i = start; i <= end; i++) {
-      writer1.append(new Key(new Text(String.format("%04d", i))), new Value(Integer.toString(i)));
-    }
-    writer1.close();
-  }
-
-  private void verifyData(AccumuloClient client, String table, int s, int e) throws Exception {
-    try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
-
-      Iterator<Entry<Key,Value>> iter = scanner.iterator();
-
-      for (int i = s; i <= e; i++) {
-        if (!iter.hasNext()) {
-          throw new Exception("row " + i + " not found");
-        }
-
-        Entry<Key,Value> entry = iter.next();
-
-        String row = String.format("%04d", i);
-
-        if (!entry.getKey().getRow().equals(new Text(row))) {
-          throw new Exception("unexpected row " + entry.getKey() + " " + i);
-        }
-
-        if (Integer.parseInt(entry.getValue().toString()) != i) {
-          throw new Exception("unexpected value " + entry + " " + i);
-        }
-      }
-
-      if (iter.hasNext()) {
-        throw new Exception("found more than expected " + iter.next());
-      }
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
index 9fd60d0..5263c88 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationIT.java
@@ -18,10 +18,9 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.time.Duration;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -41,7 +40,7 @@
 
 /**
  * This test verifies that when a lot of files are bulk imported into a table with one tablet and
- * then splits that not all map files go to the children tablets.
+ * then splits that not all data files go to the children tablets.
  */
 public class BulkSplitOptimizationIT extends AccumuloClusterHarness {
 
@@ -106,11 +105,11 @@
       // initiate splits
       c.tableOperations().setProperty(tableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "100K");
 
-      sleepUninterruptibly(2, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(2));
 
       // wait until over split threshold -- should be 78 splits
       while (c.tableOperations().listSplits(tableName).size() < 75) {
-        sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+        Thread.sleep(500);
       }
 
       FunctionalTestUtils.checkSplits(c, tableName, 50, 100);
@@ -122,7 +121,7 @@
       params.cols = 1;
       VerifyIngest.verifyIngest(c, params);
 
-      // ensure each tablet does not have all map files, should be ~2.5 files per tablet
+      // ensure each tablet does not have all data files, should be ~2.5 files per tablet
       FunctionalTestUtils.checkRFiles(c, tableName, 50, 100, 1, 4);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
index 0ba1559..be17ed5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestReader.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.test.functional;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -28,7 +27,6 @@
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
@@ -88,7 +86,7 @@
       fos.close();
       oos.close();
 
-      sleepUninterruptibly(20, TimeUnit.MILLISECONDS);
+      Thread.sleep(20);
     }
 
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
index c52bc42..7bf485f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
@@ -19,8 +19,7 @@
 package org.apache.accumulo.test.functional;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.apache.accumulo.harness.AccumuloITBase.random;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -30,7 +29,6 @@
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
@@ -71,15 +69,15 @@
       Map<String,String> expectedData = null;
       // change children in dir
 
-      for (int u = 0; u < random.nextInt(4) + 1; u++) {
+      for (int u = 0; u < RANDOM.get().nextInt(4) + 1; u++) {
         expectedData = new TreeMap<>();
 
-        if (random.nextFloat() < .5) {
+        if (RANDOM.get().nextFloat() < .5) {
           String child = UUID.randomUUID().toString();
           zk.putPersistentData(rootDir + "/dir/" + child, new byte[0], NodeExistsPolicy.SKIP);
           children.add(child);
         } else if (!children.isEmpty()) {
-          int index = random.nextInt(children.size());
+          int index = RANDOM.get().nextInt(children.size());
           String child = children.remove(index);
           zk.recursiveDelete(rootDir + "/dir/" + child, NodeMissingPolicy.FAIL);
         }
@@ -90,15 +88,15 @@
 
         // change values
         for (int i = 0; i < numData; i++) {
-          byte[] data = Long.toString(random.nextLong(), 16).getBytes(UTF_8);
+          byte[] data = Long.toString(RANDOM.get().nextLong(), 16).getBytes(UTF_8);
           zk.putPersistentData(rootDir + "/data" + i, data, NodeExistsPolicy.OVERWRITE);
           expectedData.put(rootDir + "/data" + i, new String(data, UTF_8));
         }
 
         // test a data node that does not always exists...
-        if (random.nextFloat() < .5) {
+        if (RANDOM.get().nextFloat() < .5) {
 
-          byte[] data = Long.toString(random.nextLong(), 16).getBytes(UTF_8);
+          byte[] data = Long.toString(RANDOM.get().nextLong(), 16).getBytes(UTF_8);
 
           if (dataSExists) {
             zk.putPersistentData(rootDir + "/dataS", data, NodeExistsPolicy.OVERWRITE);
@@ -162,7 +160,7 @@
           }
         }
 
-        sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
+        Thread.sleep(5);
       }
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
deleted file mode 100644
index 2ff0ecd..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/ClassLoaderIT.java
+++ /dev/null
@@ -1,121 +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.test.functional;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assumptions.assumeTrue;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.time.Duration;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.iterators.Combiner;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
-import org.junit.jupiter.api.Test;
-
-@Tag(MINI_CLUSTER_ONLY)
-public class ClassLoaderIT extends AccumuloClusterHarness {
-
-  private static final long ZOOKEEPER_PROPAGATION_TIME = 10_000;
-
-  private String rootPath;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(2);
-  }
-
-  @BeforeEach
-  public void checkCluster() {
-    assumeTrue(getClusterType() == ClusterType.MINI);
-    MiniAccumuloClusterImpl mac = (MiniAccumuloClusterImpl) getCluster();
-    rootPath = mac.getConfig().getDir().getAbsolutePath();
-  }
-
-  private static void copyStreamToFileSystem(FileSystem fs, String jarName, Path path)
-      throws IOException {
-    byte[] buffer = new byte[10 * 1024];
-    try (FSDataOutputStream dest = fs.create(path);
-        InputStream stream = ClassLoaderIT.class.getResourceAsStream(jarName)) {
-      while (true) {
-        int n = stream.read(buffer, 0, buffer.length);
-        if (n <= 0) {
-          break;
-        }
-        dest.write(buffer, 0, n);
-      }
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-      c.tableOperations().create(tableName);
-      try (BatchWriter bw = c.createBatchWriter(tableName)) {
-        Mutation m = new Mutation("row1");
-        m.put("cf", "col1", "Test");
-        bw.addMutation(m);
-      }
-      scanCheck(c, tableName, "Test");
-      FileSystem fs = getCluster().getFileSystem();
-      Path jarPath = new Path(rootPath + "/lib/ext/Test.jar");
-      copyStreamToFileSystem(fs, "/org/apache/accumulo/test/TestCombinerX.jar", jarPath);
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
-      IteratorSetting is = new IteratorSetting(10, "TestCombiner",
-          "org.apache.accumulo.test.functional.TestCombiner");
-      Combiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("cf")));
-      c.tableOperations().attachIterator(tableName, is, EnumSet.of(IteratorScope.scan));
-      sleepUninterruptibly(ZOOKEEPER_PROPAGATION_TIME, TimeUnit.MILLISECONDS);
-      scanCheck(c, tableName, "TestX");
-      fs.delete(jarPath, true);
-      copyStreamToFileSystem(fs, "/org/apache/accumulo/test/TestCombinerY.jar", jarPath);
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
-      scanCheck(c, tableName, "TestY");
-      fs.delete(jarPath, true);
-    }
-  }
-
-  private void scanCheck(AccumuloClient c, String tableName, String expected) throws Exception {
-    try (Scanner bs = c.createScanner(tableName, Authorizations.EMPTY)) {
-      String actual = getOnlyElement(bs).getValue().toString();
-      assertEquals(expected, actual);
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java
index 7c7a231..4d94cd8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java
@@ -35,7 +35,8 @@
 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.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -88,11 +89,11 @@
       // create a fake _tmp file in its directory
       String id = c.tableOperations().tableIdMap().get(tableName);
       Path file;
-      try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.setRange(Range.prefix(id));
         s.fetchColumnFamily(DataFileColumnFamily.NAME);
         Entry<Key,Value> entry = getOnlyElement(s);
-        file = new Path(entry.getKey().getColumnQualifier().toString());
+        file = StoredTabletFile.of(entry.getKey().getColumnQualifier()).getPath();
       }
 
       FileSystem fs = getCluster().getFileSystem();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.java
deleted file mode 100644
index 307b49e..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/CleanUpIT.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.test.functional;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.time.Duration;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-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.security.Authorizations;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.harness.SharedMiniClusterBase;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Iterables;
-
-/**
- * Ensures that all threads spawned for ZooKeeper and Thrift connectivity are reaped after calling
- * CleanUp.shutdown().
- *
- * Because this is destructive across the current context classloader, the normal teardown methods
- * will fail (because they attempt to create a Connector). Until the ZooKeeperInstance and Connector
- * are self-contained WRT resource management, we can't leverage the AccumuloClusterBase.
- */
-public class CleanUpIT extends SharedMiniClusterBase {
-  private static final Logger log = LoggerFactory.getLogger(CleanUpIT.class);
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofSeconds(30);
-  }
-
-  @BeforeAll
-  public static void setup() throws Exception {
-    SharedMiniClusterBase.startMiniCluster();
-  }
-
-  @AfterAll
-  public static void teardown() {
-    SharedMiniClusterBase.stopMiniCluster();
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void run() throws Exception {
-
-    // CleanUp for Connectors will not work if there are active AccumuloClients
-    assertEquals(0, SingletonManager.getReservationCount());
-
-    // CleanUp was created to clean up after connectors. This test intentionally creates a connector
-    // instead of an AccumuloClient
-    org.apache.accumulo.core.client.Connector conn =
-        new org.apache.accumulo.core.client.ZooKeeperInstance(getCluster().getInstanceName(),
-            getCluster().getZooKeepers()).getConnector(getPrincipal(), getToken());
-
-    String tableName = getUniqueNames(1)[0];
-    conn.tableOperations().create(tableName);
-
-    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
-
-    Mutation m1 = new Mutation("r1");
-    m1.put("cf1", "cq1", 1, "5");
-
-    bw.addMutation(m1);
-
-    bw.flush();
-
-    try (Scanner scanner = conn.createScanner(tableName, new Authorizations())) {
-
-      int count = 0;
-      for (Entry<Key,Value> entry : scanner) {
-        count++;
-        if (!entry.getValue().toString().equals("5")) {
-          fail("Unexpected value " + entry.getValue());
-        }
-      }
-
-      assertEquals(1, count, "Unexpected count");
-
-      int threadCount = countThreads();
-      if (threadCount < 2) {
-        printThreadNames();
-        fail("Not seeing expected threads. Saw " + threadCount);
-      }
-
-      // explicitly close the scanner to verify that the scanner throws after close when iterated
-      scanner.close();
-      assertThrows(IllegalStateException.class, () -> Iterables.size(scanner));
-    }
-
-    // close the scanners before closing the client, because the scanners need the client's cleanup
-    // thread pool to execute their cleanup tasks when they are closed, so they don't block
-    org.apache.accumulo.core.util.CleanUp.shutdownNow(conn);
-
-    Mutation m2 = new Mutation("r2");
-    m2.put("cf1", "cq1", 1, "6");
-
-    bw.addMutation(m1);
-    assertThrows(MutationsRejectedException.class, bw::flush);
-
-    // expect this to fail also, want to clean up batch writer threads
-    assertThrows(MutationsRejectedException.class, bw::close);
-
-    var threadCount = countThreads();
-    if (threadCount > 0) {
-      printThreadNames();
-      fail("Threads did not go away. Saw " + threadCount);
-    }
-  }
-
-  private void printThreadNames() {
-    Set<Thread> threads = Thread.getAllStackTraces().keySet();
-    Exception e = new Exception();
-    for (Thread thread : threads) {
-      e.setStackTrace(thread.getStackTrace());
-      log.info("thread name: " + thread.getName(), e);
-    }
-  }
-
-  /**
-   * count threads that should be cleaned up
-   *
-   */
-  private int countThreads() {
-    int count = 0;
-    Set<Thread> threads = Thread.getAllStackTraces().keySet();
-    for (Thread thread : threads) {
-
-      if (thread.getName().toLowerCase().contains("sendthread")
-          || thread.getName().toLowerCase().contains("eventthread")) {
-        count++;
-      }
-
-      if (thread.getName().toLowerCase().contains("thrift")
-          && thread.getName().toLowerCase().contains("pool")) {
-        count++;
-      }
-    }
-
-    return count;
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
index 6390a2c..06b8adc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CloneTestIT.java
@@ -56,8 +56,8 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 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 org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
@@ -137,7 +137,8 @@
   }
 
   private void checkMetadata(String table, AccumuloClient client) throws Exception {
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner s =
+        client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
 
       s.fetchColumnFamily(DataFileColumnFamily.NAME);
       ServerColumnFamily.DIRECTORY_COLUMN.fetch(s);
@@ -158,7 +159,7 @@
         k.getColumnQualifier(cq);
 
         if (cf.equals(DataFileColumnFamily.NAME)) {
-          Path p = new Path(cq.toString());
+          Path p = StoredTabletFile.of(cq).getPath();
           FileSystem fs = cluster.getFileSystem();
           assertTrue(fs.exists(p), "File does not exist: " + p);
         } else if (cf.equals(ServerColumnFamily.DIRECTORY_COLUMN.getColumnFamily())) {
@@ -338,16 +339,16 @@
   @Test
   public void testCloneRootTable() {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      assertThrows(AccumuloException.class,
-          () -> client.tableOperations().clone(RootTable.NAME, "rc1", CloneConfiguration.empty()));
+      assertThrows(AccumuloException.class, () -> client.tableOperations()
+          .clone(AccumuloTable.ROOT.tableName(), "rc1", CloneConfiguration.empty()));
     }
   }
 
   @Test
   public void testCloneMetadataTable() {
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      assertThrows(AccumuloException.class, () -> client.tableOperations().clone(MetadataTable.NAME,
-          "mc1", CloneConfiguration.empty()));
+      assertThrows(AccumuloException.class, () -> client.tableOperations()
+          .clone(AccumuloTable.METADATA.tableName(), "mc1", CloneConfiguration.empty()));
     }
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java
index 146fbee..e51b6c3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactLocationModeIT.java
@@ -32,17 +32,21 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.manager.state.MetaDataTableScanner;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 
 public class CompactLocationModeIT extends ConfigurableMacBase {
 
+  @SuppressWarnings("deprecation")
+  private static final Property DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY =
+      Property.TSERV_LAST_LOCATION_MODE;
+
   @Override
   protected Duration defaultTimeout() {
     return Duration.ofMinutes(2);
@@ -50,7 +54,7 @@
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration fsConf) {
-    cfg.setProperty(Property.TSERV_LAST_LOCATION_MODE, "compaction");
+    cfg.setProperty(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY, "compaction");
   }
 
   @Test
@@ -61,11 +65,9 @@
       c.tableOperations().create(tableName);
       String tableId = c.tableOperations().tableIdMap().get(tableName);
       // wait for the table to be online
-      TabletLocationState newTablet;
-      do {
-        UtilWaitThread.sleep(250);
-        newTablet = getTabletLocationState(c, tableId);
-      } while (newTablet.current == null);
+      Wait.waitFor(() -> getTabletLocationState(c, tableId) != null);
+
+      TabletLocationState newTablet = getTabletLocationState(c, tableId);
       assertNull(newTablet.last);
       assertNull(newTablet.future);
 
@@ -77,7 +79,7 @@
       }
       // assert that the default mode is "compact"
       assertEquals("compaction", c.instanceOperations().getSystemConfiguration()
-          .get(Property.TSERV_LAST_LOCATION_MODE.getKey()));
+          .get(DEPRECATED_TSERV_LAST_LOCATION_MODE_PROPERTY.getKey()));
 
       // no last location should be set yet
       TabletLocationState unflushed = getTabletLocationState(c, tableId);
@@ -111,7 +113,8 @@
 
   private TabletLocationState getTabletLocationState(AccumuloClient c, String tableId) {
     try (MetaDataTableScanner s = new MetaDataTableScanner((ClientContext) c,
-        new Range(TabletsSection.encodeRow(TableId.of(tableId), null)), MetadataTable.NAME)) {
+        new Range(TabletsSection.encodeRow(TableId.of(tableId), null)),
+        AccumuloTable.METADATA.tableName())) {
       return s.next();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
index 46aa609..18fdff7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -19,23 +19,33 @@
 package org.apache.accumulo.test.functional;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 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.NoSuchElementException;
 import java.util.Objects;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -53,6 +63,7 @@
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.PluginConfig;
 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.admin.compaction.CompressionConfigurer;
 import org.apache.accumulo.core.clientImpl.ClientContext;
@@ -62,6 +73,7 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.rfile.bcfile.PrintBCInfo;
 import org.apache.accumulo.core.iterators.DevNull;
 import org.apache.accumulo.core.iterators.Filter;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
@@ -69,7 +81,8 @@
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.user.AgeOffFilter;
 import org.apache.accumulo.core.iterators.user.GrepIterator;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
@@ -81,6 +94,7 @@
 import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
@@ -99,7 +113,6 @@
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 
-@SuppressWarnings("removal")
 public class CompactionIT extends AccumuloClusterHarness {
 
   public static class TestFilter extends Filter {
@@ -164,6 +177,67 @@
 
   }
 
+  /**
+   * CompactionSelector that selects nothing for testing
+   */
+  public static class EmptyCompactionSelector implements CompactionSelector {
+
+    @Override
+    public void init(InitParameters iparams) {
+
+    }
+
+    @Override
+    public Selection select(SelectionParameters sparams) {
+      return new Selection(Set.of());
+    }
+  }
+
+  /**
+   * A CompactionConfigurer that can be used to configure the compression type used for intermediate
+   * and final compactions. An intermediate compaction is a compaction whose result is short-lived.
+   * For instance, if 10 files are selected for compaction, 5 of these files are compacted to a file
+   * 'f0', then f0 is compacted with the 5 remaining files creating file 'f1', then f0 would be an
+   * intermediate file and f1 would be the final file.
+   */
+  public static class CompressionTypeConfigurer implements CompactionConfigurer {
+    public static final String COMPRESS_TYPE_KEY = Property.TABLE_FILE_COMPRESSION_TYPE.getKey();
+    public static final String FINAL_COMPRESS_TYPE_KEY = "final.compress.type";
+    public static final String INTERMEDIATE_COMPRESS_TYPE_KEY = "intermediate.compress.type";
+    private String finalCompressTypeVal;
+    private String interCompressTypeVal;
+
+    @Override
+    public void init(InitParameters iparams) {
+      var options = iparams.getOptions();
+      String finalCompressTypeVal = options.get(FINAL_COMPRESS_TYPE_KEY);
+      String interCompressTypeVal = options.get(INTERMEDIATE_COMPRESS_TYPE_KEY);
+      if (finalCompressTypeVal != null && interCompressTypeVal != null) {
+        this.finalCompressTypeVal = finalCompressTypeVal;
+        this.interCompressTypeVal = interCompressTypeVal;
+      } else {
+        throw new IllegalArgumentException(
+            "Must set " + FINAL_COMPRESS_TYPE_KEY + " and " + INTERMEDIATE_COMPRESS_TYPE_KEY);
+      }
+    }
+
+    @Override
+    public Overrides override(InputParameters params) {
+      var inputFiles = params.getInputFiles();
+      var selectedFiles = params.getSelectedFiles();
+      // If this is the final compaction, set the compression type to the value set for
+      // finalCompressTypeVal
+      // If this is an intermediate compaction, set the compression type to the value set for
+      // interCompressTypeVal
+      if (selectedFiles.equals(inputFiles instanceof Set ? inputFiles : Set.copyOf(inputFiles))) {
+        return new Overrides(Map.of(COMPRESS_TYPE_KEY, finalCompressTypeVal));
+      } else {
+        return new Overrides(Map.of(COMPRESS_TYPE_KEY, interCompressTypeVal));
+      }
+    }
+
+  }
+
   private static final Logger log = LoggerFactory.getLogger(CompactionIT.class);
 
   private static final int MAX_DATA = 1000;
@@ -176,9 +250,7 @@
   @Override
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.TSERV_MAJC_THREAD_MAXOPEN, "4");
     cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
-    cfg.setProperty(Property.TSERV_MAJC_MAXCONCURRENT, "1");
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
@@ -308,18 +380,22 @@
       ReadWriteIT.ingest(client, MAX_DATA, 1, 1, 0, "colf", table1, 1);
 
       Ample ample = ((ClientContext) client).getAmple();
-      TabletsMetadata tms = ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build();
-      TabletMetadata tm = tms.iterator().next();
-      assertEquals(1000, tm.getFiles().size());
+      try (TabletsMetadata tms =
+          ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build();) {
+        TabletMetadata tm = tms.iterator().next();
+        assertEquals(1000, tm.getFiles().size());
+      }
 
       IteratorSetting setting = new IteratorSetting(50, "error", ErrorThrowingIterator.class);
       setting.addOption(ErrorThrowingIterator.TIMES, "3");
       client.tableOperations().attachIterator(table1, setting, EnumSet.of(IteratorScope.majc));
       client.tableOperations().compact(table1, new CompactionConfig().setWait(true));
 
-      tms = ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build();
-      tm = tms.iterator().next();
-      assertEquals(1, tm.getFiles().size());
+      try (
+          TabletsMetadata tms = ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build()) {
+        TabletMetadata tm = tms.iterator().next();
+        assertEquals(1, tm.getFiles().size());
+      }
 
       ReadWriteIT.verify(client, MAX_DATA, 1, 1, 0, table1);
 
@@ -339,9 +415,11 @@
       ReadWriteIT.verify(client, 50, 1, 1, 0, table1);
 
       Ample ample = ((ClientContext) client).getAmple();
-      TabletsMetadata tms = ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build();
-      TabletMetadata tm = tms.iterator().next();
-      assertEquals(50, tm.getFiles().size());
+      try (
+          TabletsMetadata tms = ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build()) {
+        TabletMetadata tm = tms.iterator().next();
+        assertEquals(50, tm.getFiles().size());
+      }
 
       IteratorSetting setting = new IteratorSetting(50, "ageoff", AgeOffFilter.class);
       setting.addOption("ttl", "0");
@@ -355,8 +433,9 @@
       client.tableOperations().attachIterator(table1, setting2, EnumSet.of(IteratorScope.majc));
       client.tableOperations().compact(table1, new CompactionConfig().setWait(true));
 
-      assertThrows(NoSuchElementException.class, () -> ample.readTablets().forTable(tid)
-          .fetch(ColumnType.FILES).build().iterator().next());
+      try (TabletsMetadata tm = ample.readTablets().forTable(tid).fetch(ColumnType.FILES).build()) {
+        assertThrows(NoSuchElementException.class, () -> tm.iterator().next());
+      }
       assertEquals(0, client.createScanner(table1).stream().count());
     }
   }
@@ -630,6 +709,198 @@
   }
 
   @Test
+  public void testSelectNoFiles() throws Exception {
+
+    // Test a compaction selector that selects no files. In this case there is no work to,
+    // so we want to ensure it does not hang
+
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      c.tableOperations().create(tableName);
+
+      writeFlush(c, tableName, "a");
+      writeFlush(c, tableName, "b");
+
+      CompactionConfig config = new CompactionConfig()
+          .setSelector(new PluginConfig(EmptyCompactionSelector.class.getName(), Map.of()))
+          .setWait(true);
+      c.tableOperations().compact(tableName, config);
+
+      assertEquals(Set.of("a", "b"), getRows(c, tableName));
+    }
+
+  }
+
+  @Test
+  public void testConcurrent() throws Exception {
+    // two compactions without iterators or strategy should be able to run concurrently
+
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+
+      String tableName = getUniqueNames(1)[0];
+      c.tableOperations().create(tableName);
+
+      // write random data because its very unlikely it will compress
+      writeRandomValue(c, tableName, 1 << 16);
+      writeRandomValue(c, tableName, 1 << 16);
+
+      c.tableOperations().compact(tableName, new CompactionConfig().setWait(false));
+      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+
+      assertEquals(1, FunctionalTestUtils.countRFiles(c, tableName));
+
+      writeRandomValue(c, tableName, 1 << 16);
+
+      IteratorSetting iterConfig = new IteratorSetting(30, SlowIterator.class);
+      SlowIterator.setSleepTime(iterConfig, 1000);
+
+      long t1 = System.currentTimeMillis();
+      c.tableOperations().compact(tableName,
+          new CompactionConfig().setWait(false).setIterators(java.util.Arrays.asList(iterConfig)));
+      try {
+        // this compaction should fail because previous one set iterators
+        c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+        if (System.currentTimeMillis() - t1 < 2000) {
+          fail("Expected compaction to fail because another concurrent compaction set iterators");
+        }
+      } catch (AccumuloException e) {}
+    }
+  }
+
+  @Test
+  public void testGetSelectedFilesForCompaction() throws Exception {
+
+    // Tests CompactionConfigurer.InputParameters.getSelectedFiles()
+
+    String tableName = this.getUniqueNames(1)[0];
+    // Disable GC so intermediate compaction files are not deleted
+    getCluster().getClusterControl().stopAllServers(ServerType.GARBAGE_COLLECTOR);
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      Map<String,String> props = new HashMap<>();
+      props.put(Property.TABLE_FILE_COMPRESSION_TYPE.getKey(), "none");
+      // This is done to avoid system compactions - we want to do all the compactions ourselves
+      props.put("table.compaction.dispatcher.opts.service.system", "nonexitant");
+      NewTableConfiguration ntc = new NewTableConfiguration().setProperties(props);
+      client.tableOperations().create(tableName, ntc);
+
+      // The following will create 4 small and 4 large RFiles
+      // The 4 small files will be compacted into one file (an "intermediate compaction" file)
+      // Then, this file will be compacted with the 4 large files, creating the final compaction
+      // file
+      byte[] largeData = new byte[1_000_000];
+      byte[] smallData = new byte[100_000];
+      final int numFiles = 8;
+      Arrays.fill(largeData, (byte) 65);
+      Arrays.fill(smallData, (byte) 65);
+      try (var writer = client.createBatchWriter(tableName)) {
+        for (int i = 0; i < numFiles; i++) {
+          Mutation mut = new Mutation("r" + i);
+          if (i < numFiles / 2) {
+            mut.at().family("f").qualifier("q").put(largeData);
+          } else {
+            mut.at().family("f").qualifier("q").put(smallData);
+          }
+          writer.addMutation(mut);
+          writer.flush();
+          client.tableOperations().flush(tableName, null, null, true);
+        }
+      }
+
+      client.tableOperations().compact(tableName,
+          new CompactionConfig().setWait(true)
+              .setConfigurer(new PluginConfig(CompressionTypeConfigurer.class.getName(),
+                  Map.of(CompressionTypeConfigurer.FINAL_COMPRESS_TYPE_KEY, "snappy",
+                      CompressionTypeConfigurer.INTERMEDIATE_COMPRESS_TYPE_KEY, "gz"))));
+
+      var tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
+      // The directory of the RFiles
+      java.nio.file.Path rootPath = null;
+      // The path to the final compaction RFile (located within rootPath)
+      java.nio.file.Path finalCompactionFilePath = null;
+      int count = 0;
+      try (var tabletsMeta =
+          TabletsMetadata.builder(client).forTable(tableId).fetch(ColumnType.FILES).build()) {
+        for (TabletMetadata tm : tabletsMeta) {
+          for (StoredTabletFile stf : tm.getFiles()) {
+            // Since the 8 files should be compacted down to 1 file, these should only be set once
+            finalCompactionFilePath = Paths.get(stf.getPath().toUri().getRawPath());
+            rootPath = Paths.get(stf.getPath().getParent().toUri().getRawPath());
+            count++;
+          }
+        }
+      }
+      assertEquals(1, count);
+      assertNotNull(finalCompactionFilePath);
+      assertNotNull(rootPath);
+      String finalCompactionFile = finalCompactionFilePath.toString();
+      // The following will find the intermediate compaction file in the root path.
+      // Intermediate compaction files begin with 'C' and end with '.rf'
+      final String[] interCompactionFile = {null};
+      Files.walkFileTree(rootPath, new SimpleFileVisitor<java.nio.file.Path>() {
+        @Override
+        public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs)
+            throws IOException {
+          String regex = "^C.*\\.rf$";
+          java.nio.file.Path fileName = (file != null) ? file.getFileName() : null;
+          if (fileName != null && fileName.toString().matches(regex)) {
+            interCompactionFile[0] = file.toString();
+            return FileVisitResult.TERMINATE;
+          }
+          return FileVisitResult.CONTINUE;
+        }
+      });
+      assertNotNull(interCompactionFile[0]);
+      String[] args = new String[3];
+      args[0] = "--props";
+      args[1] = getCluster().getAccumuloPropertiesPath();
+      args[2] = finalCompactionFile;
+      PrintBCInfo bcInfo = new PrintBCInfo(args);
+      String finalCompressionType = bcInfo.getCompressionType();
+      // The compression type used on the final compaction file should be 'snappy'
+      assertEquals("snappy", finalCompressionType);
+      args[2] = interCompactionFile[0];
+      bcInfo = new PrintBCInfo(args);
+      String interCompressionType = bcInfo.getCompressionType();
+      // The compression type used on the intermediate compaction file should be 'gz'
+      assertEquals("gz", interCompressionType);
+    } finally {
+      // Re-enable GC
+      getCluster().getClusterControl().startAllServers(ServerType.GARBAGE_COLLECTOR);
+    }
+  }
+
+  /**
+   * Was used in debugging {@link #testGetSelectedFilesForCompaction}. May be useful later.
+   *
+   * @param client An accumulo client
+   * @param tableName The name of the table
+   * @return a map of the RFiles to their size in bytes
+   */
+  private Map<String,Long> getFileSizeMap(AccumuloClient client, String tableName) {
+    var tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
+    Map<String,Long> map = new HashMap<>();
+
+    try (var tabletsMeta =
+        TabletsMetadata.builder(client).forTable(tableId).fetch(ColumnType.FILES).build()) {
+      for (TabletMetadata tm : tabletsMeta) {
+        for (StoredTabletFile stf : tm.getFiles()) {
+          try {
+            String filePath = stf.getPath().toString();
+            Long fileSize =
+                FileSystem.getLocal(new Configuration()).getFileStatus(stf.getPath()).getLen();
+            map.put(filePath, fileSize);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        }
+      }
+
+      return map;
+    }
+  }
+
+  @Test
   public void testDeleteCompactionService() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       var uniqueNames = getUniqueNames(2);
@@ -638,18 +909,18 @@
 
       // create a compaction service named deleteme
       c.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner",
           DefaultCompactionPlanner.class.getName());
       c.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner.opts.executors",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner.opts.executors",
           "[{'name':'all','type':'internal','numThreads':1}]".replaceAll("'", "\""));
 
       // create a compaction service named keepme
       c.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "keepme.planner",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "keepme.planner",
           DefaultCompactionPlanner.class.getName());
       c.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "keepme.planner.opts.executors",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "keepme.planner.opts.executors",
           "[{'name':'all','type':'internal','numThreads':1}]".replaceAll("'", "\""));
 
       // create a table that uses the compaction service deleteme
@@ -680,16 +951,16 @@
 
       // delete the compaction service deleteme
       c.instanceOperations()
-          .removeProperty(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner");
+          .removeProperty(Property.COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner");
       c.instanceOperations().removeProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner.opts.executors");
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "deleteme.planner.opts.executors");
 
       // add a new compaction service named newcs
       c.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner",
           DefaultCompactionPlanner.class.getName());
       c.instanceOperations().setProperty(
-          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.executors",
+          Property.COMPACTION_SERVICE_PREFIX.getKey() + "newcs.planner.opts.executors",
           "[{'name':'all','type':'internal','numThreads':1}]".replaceAll("'", "\""));
 
       // set table 1 to a compaction service newcs
@@ -720,7 +991,7 @@
    */
   private int countFiles(AccumuloClient c, String tableName) throws Exception {
     var tableId = getCluster().getServerContext().getTableId(tableName);
-    try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       s.setRange(MetadataSchema.TabletsSection.getRange(tableId));
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(s);
       s.fetchColumnFamily(new Text(DataFileColumnFamily.NAME));
@@ -728,4 +999,35 @@
     }
   }
 
+  private void writeRandomValue(AccumuloClient c, String tableName, int size) throws Exception {
+    byte[] data1 = new byte[size];
+    RANDOM.get().nextBytes(data1);
+
+    try (BatchWriter bw = c.createBatchWriter(tableName)) {
+      Mutation m1 = new Mutation("r" + RANDOM.get().nextInt(909090));
+      m1.put("data", "bl0b", new Value(data1));
+      bw.addMutation(m1);
+    }
+    c.tableOperations().flush(tableName, null, null, true);
+  }
+
+  private Set<String> getRows(AccumuloClient c, String tableName) throws TableNotFoundException {
+    Set<String> rows = new HashSet<>();
+    try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
+      for (Entry<Key,Value> entry : scanner) {
+        rows.add(entry.getKey().getRowData().toString());
+      }
+    }
+    return rows;
+  }
+
+  private void writeFlush(AccumuloClient client, String tablename, String row) throws Exception {
+    try (BatchWriter bw = client.createBatchWriter(tablename)) {
+      Mutation m = new Mutation(row);
+      m.put("", "", "");
+      bw.addMutation(m);
+    }
+    client.tableOperations().flush(tablename, null, null, true);
+  }
+
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
index 67183ef..9fe5ae1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrencyIT.java
@@ -18,12 +18,9 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
 import java.time.Duration;
 import java.util.EnumSet;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -123,7 +120,7 @@
     ScanTask st1 = new ScanTask(c, tableName, 100);
     st1.start();
 
-    sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+    Thread.sleep(50);
     c.tableOperations().flush(tableName, null, null, true);
 
     for (int i = 0; i < 50; i++) {
@@ -150,7 +147,7 @@
     ScanTask st3 = new ScanTask(c, tableName, 150);
     st3.start();
 
-    sleepUninterruptibly(50, TimeUnit.MILLISECONDS);
+    Thread.sleep(50);
     c.tableOperations().flush(tableName, null, null, false);
 
     st3.join();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java
index f1ab650..fab780f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConcurrentDeleteTableIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
@@ -267,7 +268,7 @@
       throws TableNotFoundException, MutationsRejectedException {
     try (BatchWriter bw = c.createBatchWriter(table)) {
       for (int i = 0; i < 1_000; i++) {
-        Mutation m = new Mutation(String.format("%09x", random.nextInt(100_000_000)));
+        Mutation m = new Mutation(String.format("%09x", RANDOM.get().nextInt(100_000_000)));
         m.put("m", "order", "" + i);
         bw.addMutation(m);
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java
deleted file mode 100644
index f91db86..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConfigurableCompactionIT.java
+++ /dev/null
@@ -1,183 +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.test.functional;
-
-import static java.util.Collections.singletonMap;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Iterators;
-
-public class ConfigurableCompactionIT extends ConfigurableMacBase {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(2);
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setSiteConfig(singletonMap(Property.TSERV_MAJC_DELAY.getKey(), "1s"));
-  }
-
-  @SuppressWarnings("removal")
-  public static class SimpleCompactionStrategy
-      extends org.apache.accumulo.tserver.compaction.CompactionStrategy {
-
-    @Override
-    public void init(Map<String,String> options) {
-      String countString = options.get("count");
-      if (countString != null) {
-        count = Integer.parseInt(countString);
-      }
-    }
-
-    int count = 3;
-
-    @Override
-    public boolean
-        shouldCompact(org.apache.accumulo.tserver.compaction.MajorCompactionRequest request) {
-      return request.getFiles().size() == count;
-
-    }
-
-    @Override
-    public org.apache.accumulo.tserver.compaction.CompactionPlan
-        getCompactionPlan(org.apache.accumulo.tserver.compaction.MajorCompactionRequest request) {
-      var result = new org.apache.accumulo.tserver.compaction.CompactionPlan();
-      result.inputFiles.addAll(request.getFiles().keySet());
-      return result;
-    }
-
-  }
-
-  @SuppressWarnings("removal")
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-      final String tableName = getUniqueNames(1)[0];
-
-      c.tableOperations().create(tableName, new NewTableConfiguration().setProperties(singletonMap(
-          Property.TABLE_COMPACTION_STRATEGY.getKey(), SimpleCompactionStrategy.class.getName())));
-      runTest(c, tableName, 3);
-
-      c.tableOperations().setProperty(tableName,
-          Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey() + "count", "" + 5);
-      runTest(c, tableName, 5);
-    }
-  }
-
-  @SuppressWarnings("removal")
-  @Test
-  public void testPerTableClasspath() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-      final String tableName = getUniqueNames(1)[0];
-      var destFile = initJar("/org/apache/accumulo/test/TestCompactionStrat.jar",
-          "TestCompactionStrat", getCluster().getConfig().getAccumuloDir().getAbsolutePath());
-      c.instanceOperations().setProperty(
-          Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "context1", destFile.toString());
-      Map<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_MAJC_RATIO.getKey(), "10");
-      props.put(Property.TABLE_CLASSLOADER_CONTEXT.getKey(), "context1");
-      // EfgCompactionStrat will only compact a tablet w/ end row of 'efg'. No other tablets are
-      // compacted.
-      props.put(Property.TABLE_COMPACTION_STRATEGY.getKey(),
-          "org.apache.accumulo.test.EfgCompactionStrat");
-      c.tableOperations().create(tableName, new NewTableConfiguration().setProperties(props)
-          .withSplits(new TreeSet<>(Arrays.asList(new Text("efg")))));
-
-      for (char ch = 'a'; ch < 'l'; ch++) {
-        writeFlush(c, tableName, ch + "");
-      }
-
-      while (countFiles(c) != 7) {
-        UtilWaitThread.sleep(200);
-      }
-    }
-  }
-
-  private void writeFlush(AccumuloClient client, String tablename, String row) throws Exception {
-    try (BatchWriter bw = client.createBatchWriter(tablename)) {
-      Mutation m = new Mutation(row);
-      m.put("", "", "");
-      bw.addMutation(m);
-    }
-    client.tableOperations().flush(tablename, null, null, true);
-  }
-
-  private void makeFile(AccumuloClient client, String tablename) throws Exception {
-    try (BatchWriter bw = client.createBatchWriter(tablename)) {
-      byte[] empty = {};
-      byte[] row = new byte[10];
-      random.nextBytes(row);
-      Mutation m = new Mutation(row, 0, 10);
-      m.put(empty, empty, empty);
-      bw.addMutation(m);
-      bw.flush();
-    }
-    client.tableOperations().flush(tablename, null, null, true);
-  }
-
-  private void runTest(final AccumuloClient c, final String tableName, final int n)
-      throws Exception {
-    for (int i = countFiles(c); i < n - 1; i++) {
-      makeFile(c, tableName);
-    }
-    assertEquals(n - 1, countFiles(c));
-    makeFile(c, tableName);
-    for (int i = 0; i < 10; i++) {
-      int count = countFiles(c);
-      assertTrue(count == 1 || count == n);
-      if (count == 1) {
-        break;
-      }
-      UtilWaitThread.sleep(1000);
-    }
-  }
-
-  private int countFiles(AccumuloClient c) throws Exception {
-    try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.fetchColumnFamily(DataFileColumnFamily.NAME);
-      return Iterators.size(s.iterator());
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
index 9f99d7e..1695f6d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ConstraintIT.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.time.Duration;
 import java.util.HashMap;
@@ -26,7 +26,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -163,7 +162,7 @@
 
       // remove the numeric value constraint
       client.tableOperations().removeConstraint(tableName, 2);
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
 
       // now should be able to add a non numeric value
       bw = client.createBatchWriter(tableName);
@@ -190,7 +189,7 @@
       // add a constraint that references a non-existent class
       client.tableOperations().setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX + "1",
           "com.foobar.nonExistentClass");
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
 
       // add a mutation
       bw = client.createBatchWriter(tableName);
@@ -233,7 +232,7 @@
 
       // remove the bad constraint
       client.tableOperations().removeConstraint(tableName, 1);
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
 
       // try the mutation again
       bw = client.createBatchWriter(tableName);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CreateInitialSplitsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CreateInitialSplitsIT.java
index e69a305..0b9b7e6 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CreateInitialSplitsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CreateInitialSplitsIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -210,7 +211,7 @@
     SortedSet<Text> splits = new TreeSet<>();
     for (int i = 0; i < numItems; i++) {
       byte[] split = new byte[len];
-      random.nextBytes(split);
+      RANDOM.get().nextBytes(split);
       splits.add(encode(new Text(split), useB64));
     }
     return splits;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
index b70d5eb..7060f5c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
@@ -40,7 +40,7 @@
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -109,8 +109,8 @@
       try (
           AccumuloClient userAccumuloClient =
               Accumulo.newClient().from(client.properties()).as(username, token).build();
-          Scanner scanner =
-              userAccumuloClient.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+          Scanner scanner = userAccumuloClient.createScanner(AccumuloTable.METADATA.tableName(),
+              Authorizations.EMPTY)) {
         assertFalse(token.isDestroyed());
         token.destroy();
         assertTrue(token.isDestroyed());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
index 43f6da0..6140540 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteEverythingIT.java
@@ -18,12 +18,11 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -105,7 +104,7 @@
         c.tableOperations().flush(tableName, null, null, true);
 
         c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "1.0");
-        sleepUninterruptibly(4, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(4));
 
         FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 0, 0);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
index a18d43d..88768a7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.test.util.FileMetadataUtil.printAndVerifyFileMetadata;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -26,6 +27,7 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.stream.Collectors;
 
@@ -35,6 +37,8 @@
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -88,34 +92,84 @@
       String tableName = getUniqueNames(1)[0];
       testSplit(c, tableName + i++, "f", "h", "abcdefijklmnopqrstuvwxyz", 260);
       // Eliminate whole tablets, partial first tablet
-      testSplit(c, tableName + i++, "f1", "h", "abcdeff1ijklmnopqrstuvwxyz", 262);
+      testSplit(c, tableName + i++, "f1", "h", "abcdefgijklmnopqrstuvwxyz", 262);
       // Eliminate whole tablets, partial last tablet
       testSplit(c, tableName + i++, "f", "h1", "abcdefijklmnopqrstuvwxyz", 258);
       // Eliminate whole tablets, partial first and last tablet
-      testSplit(c, tableName + i++, "f1", "h1", "abcdeff1ijklmnopqrstuvwxyz", 260);
+      testSplit(c, tableName + i++, "f1", "h1", "abcdefgijklmnopqrstuvwxyz", 260);
       // Eliminate one tablet
       testSplit(c, tableName + i++, "f", "g", "abcdefhijklmnopqrstuvwxyz", 270);
+      // Eliminate first tablet
+      testSplit(c, tableName + i++, null, "a", "bcdefghijklmnopqrstuvwxyz", 270);
+      // Eliminate last tablet
+      testSplit(c, tableName + i++, "z", null, "abcdefghijklmnopqrstuvwxyz", 260);
       // Eliminate partial tablet, matches start split
       testSplit(c, tableName + i++, "f", "f1", "abcdefghijklmnopqrstuvwxyz", 278);
       // Eliminate partial tablet, matches end split
-      testSplit(c, tableName + i++, "f1", "g", "abcdeff1hijklmnopqrstuvwxyz", 272);
+      testSplit(c, tableName + i++, "f1", "g", "abcdefghijklmnopqrstuvwxyz", 272);
       // Eliminate tablets starting at -inf
       testSplit(c, tableName + i++, null, "h", "ijklmnopqrstuvwxyz", 200);
       // Eliminate tablets ending at +inf
       testSplit(c, tableName + i++, "t", null, "abcdefghijklmnopqrst", 200);
       // Eliminate some rows inside one tablet
-      testSplit(c, tableName + i++, "t0", "t2", "abcdefghijklmnopqrstt0uvwxyz", 278);
+      testSplit(c, tableName + i++, "t0", "t2", "abcdefghijklmnopqrstuvwxyz", 278);
       // Eliminate some rows in the first tablet
       testSplit(c, tableName + i++, null, "A1", "abcdefghijklmnopqrstuvwxyz", 278);
       // Eliminate some rows in the last tablet
-      testSplit(c, tableName + i++, "{1", null, "abcdefghijklmnopqrstuvwxyz{1", 272);
+      testSplit(c, tableName + i++, "{1", null, "abcdefghijklmnopqrstuvwxyz", 272);
       // Delete everything
       testSplit(c, tableName + i++, null, null, "", 0);
     }
   }
 
+  // Test that deletion works on tablets that have files that have already been fenced
+  // The fenced files are created by doing merges first
+  @Test
+  public void testManyRowsAlreadyFenced() throws Exception {
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      // Delete ranges of rows, and verify the tablets are removed.
+      int i = 0;
+      // Eliminate whole tablets
+      String tableName = getUniqueNames(1)[0];
+      testSplit(c, tableName + i++, "f", "h", "abcdefijklmnopqrstuvwxyz", 260, "f", "h");
+      // Eliminate whole tablets, partial first tablet
+      testSplit(c, tableName + i++, "f1", "h", "abcdefgijklmnopqrstuvwxyz", 262, "f", "h");
+      // Eliminate whole tablets, partial last tablet
+      testSplit(c, tableName + i++, "f", "h1", "abcdefijklmnopqrstuvwxyz", 258, "f", "h");
+      // Eliminate whole tablets, partial first and last tablet
+      testSplit(c, tableName + i++, "f1", "h1", "abcdefgijklmnopqrstuvwxyz", 260, "f", "h");
+      // Eliminate one tablet
+      testSplit(c, tableName + i++, "f", "g", "abcdefhijklmnopqrstuvwxyz", 270, "f", "g");
+      // Eliminate first tablet
+      testSplit(c, tableName + i++, null, "a", "bcdefghijklmnopqrstuvwxyz", 270, "a", "a");
+      // Eliminate last tablet
+      testSplit(c, tableName + i++, "z", null, "abcdefghijklmnopqrstuvwxyz", 260, "z", "z");
+      // Eliminate partial tablet, matches start split
+      testSplit(c, tableName + i++, "f", "f1", "abcdefghijklmnopqrstuvwxyz", 278, "f", "f");
+      // Eliminate partial tablet, matches end split
+      testSplit(c, tableName + i++, "f1", "g", "abcdefghijklmnopqrstuvwxyz", 272, "f", "g");
+      // Eliminate tablets starting at -inf
+      testSplit(c, tableName + i++, null, "h", "ijklmnopqrstuvwxyz", 200, "a", "h");
+      // Eliminate tablets ending at +inf
+      testSplit(c, tableName + i++, "t", null, "abcdefghijklmnopqrst", 200, "t", "z");
+      // Eliminate some rows inside one tablet
+      testSplit(c, tableName + i++, "t0", "t2", "abcdefghijklmnopqrstuvwxyz", 278, "t", "t");
+      // Eliminate some rows in the first tablet
+      testSplit(c, tableName + i++, null, "A1", "abcdefghijklmnopqrstuvwxyz", 278, "a", "a");
+      // Eliminate some rows in the last tablet
+      testSplit(c, tableName + i++, "{1", null, "abcdefghijklmnopqrstuvwxyz", 272, "z", "z");
+      // Delete everything
+      testSplit(c, tableName + i++, null, null, "", 0, "a", "z");
+    }
+  }
+
   private void testSplit(AccumuloClient c, String table, String start, String end, String result,
       int entries) throws Exception {
+    testSplit(c, table, start, end, result, entries, null, null);
+  }
+
+  private void testSplit(AccumuloClient c, String table, String start, String end, String result,
+      int entries, String mergeStart, String mergeEnd) throws Exception {
     // Put a bunch of rows on each tablet
     c.tableOperations().create(table);
     try (BatchWriter bw = c.createBatchWriter(table)) {
@@ -128,8 +182,41 @@
       }
       bw.flush();
     }
+
+    final TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(table));
     // Split the table
-    c.tableOperations().addSplits(table, SPLITS);
+
+    // If a merge range is defined then merge the tablets given in the range after
+    // The purpose of the merge is to generate file metadata that contains ranges
+    // so this will test deletings on existing ranged files
+    if (mergeStart != null) {
+      SortedSet<Text> splits = new TreeSet<>(SPLITS);
+      // Generate 2 split points for each existing split and add
+      SortedSet<Text> mergeSplits =
+          SPLITS.subSet(new Text(mergeStart), true, new Text(mergeEnd), true);
+      mergeSplits.forEach(split -> splits.add(new Text(split.toString() + (ROWS_PER_TABLET / 2))));
+
+      log.debug("After splits");
+      c.tableOperations().addSplits(table, splits);
+      printAndVerifyFileMetadata(getServerContext(), tableId);
+
+      // Merge back the extra splits to a single tablet per letter to generate 2 files per tablet
+      // that have a range
+      mergeSplits.forEach(split -> {
+        try {
+          c.tableOperations().merge(table, split, new Key(split.toString() + (ROWS_PER_TABLET / 2))
+              .followingKey(PartialKey.ROW).getRow());
+          log.debug("After Merge");
+          printAndVerifyFileMetadata(getServerContext(), tableId);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      });
+    } else {
+      c.tableOperations().addSplits(table, SPLITS);
+      log.debug("After splits");
+      printAndVerifyFileMetadata(getServerContext(), tableId);
+    }
 
     Text startText = start == null ? null : new Text(start);
     Text endText = end == null ? null : new Text(end);
@@ -140,7 +227,10 @@
     for (Text split : remainingSplits) {
       sb.append(split);
     }
+    log.debug("After delete");
+    printAndVerifyFileMetadata(getServerContext(), tableId);
     assertEquals(result, sb.toString());
+
     // See that the rows are really deleted
     try (Scanner scanner = c.createScanner(table, Authorizations.EMPTY)) {
       int count = 0;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
index 5a997b7..e8b04be 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.test.functional;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
@@ -29,7 +28,6 @@
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -102,7 +100,7 @@
         });
         t.start();
 
-        sleepUninterruptibly(test * 2, TimeUnit.MILLISECONDS);
+        Thread.sleep(test * 2);
 
         client.tableOperations().deleteRows(tableName, start, end);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
index 5c04cde..74a5dd8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DeletedTablesDontFlushIT.java
@@ -28,7 +28,6 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
@@ -61,7 +60,7 @@
       SlowIterator.setSleepTime(setting, 1000);
       c.tableOperations().attachIterator(tableName, setting, EnumSet.of(IteratorScope.minc));
       // let the configuration change propagate through zookeeper
-      UtilWaitThread.sleep(1000);
+      Thread.sleep(1000);
 
       Mutation m = new Mutation("xyzzy");
       for (int i = 0; i < 100; i++) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java
index 906fe2b..e9b8f03 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java
@@ -31,7 +31,7 @@
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -156,7 +156,8 @@
     try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
       String tableName = getUniqueNames(1)[0];
       c.instanceOperations().setProperty(Property.TABLE_DURABILITY.getKey(), "none");
-      Map<String,String> props = c.tableOperations().getConfiguration(MetadataTable.NAME);
+      Map<String,String> props =
+          c.tableOperations().getConfiguration(AccumuloTable.METADATA.tableName());
       assertEquals("sync", props.get(Property.TABLE_DURABILITY.getKey()));
       c.tableOperations().create(tableName);
       props = c.tableOperations().getConfiguration(tableName);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java
index e5f07c0..9cdd9d8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java
@@ -47,9 +47,9 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.AdminUtil;
 import org.apache.accumulo.core.fate.ZooStore;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.test.util.SlowOps;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
index 3f68a45..92a2d82 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FateStarvationIT.java
@@ -18,6 +18,8 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
+
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.List;
@@ -57,8 +59,8 @@
       List<Text> splits = new ArrayList<>(TestIngest.getSplitPoints(0, 100000, 67));
 
       for (int i = 0; i < 100; i++) {
-        int idx1 = random.nextInt(splits.size() - 1);
-        int idx2 = random.nextInt(splits.size() - (idx1 + 1)) + idx1 + 1;
+        int idx1 = RANDOM.get().nextInt(splits.size() - 1);
+        int idx2 = RANDOM.get().nextInt(splits.size() - (idx1 + 1)) + idx1 + 1;
 
         c.tableOperations().compact(tableName, splits.get(idx1), splits.get(idx2), false, false);
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java
new file mode 100644
index 0000000..d931c76
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java
@@ -0,0 +1,488 @@
+/*
+ * 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.test.functional;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.apache.accumulo.test.util.FileMetadataUtil.printAndVerifyFileMetadata;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.time.Duration;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Mutation;
+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.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.TestIngest.IngestParams;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.accumulo.test.VerifyIngest.VerifyParams;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.MoreCollectors;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class FileMetadataIT extends AccumuloClusterHarness {
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(6);
+  }
+
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
+  }
+
+  // private static final Logger log = LoggerFactory.getLogger(FileMetadataIT.class);
+  static final int COLS = 1;
+  static final String COLF = "colf";
+
+  public static void ingest(AccumuloClient accumuloClient, int rows, int cols, int width,
+      int offset, String tableName) throws Exception {
+    IngestParams params = new IngestParams(accumuloClient.properties(), tableName, rows);
+    params.cols = cols;
+    params.dataSize = width;
+    params.startRow = offset;
+    params.columnFamily = COLF;
+    params.createTable = true;
+    TestIngest.ingest(accumuloClient, params);
+  }
+
+  private static void verify(AccumuloClient accumuloClient, int rows, int cols, int width,
+      int offset, String tableName) throws Exception {
+    VerifyParams params = new VerifyParams(accumuloClient.properties(), tableName, rows);
+    params.rows = rows;
+    params.dataSize = width;
+    params.startRow = offset;
+    params.columnFamily = COLF;
+    params.cols = cols;
+    VerifyIngest.verifyIngest(accumuloClient, params);
+  }
+
+  public static Text t(String s) {
+    return new Text(s);
+  }
+
+  public static Mutation m(String row, String cf, String cq, String value) {
+    Mutation m = new Mutation(t(row));
+    m.put(t(cf), t(cq), new Value(value));
+    return m;
+  }
+
+  @Test
+  public void contiguousRangeTest() throws Exception {
+    ServerContext ctx = getCluster().getServerContext();
+
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      // Need permission to write to metadata
+      accumuloClient.securityOperations().grantTablePermission(accumuloClient.whoami(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+
+      final int rows = 10000;
+      final String tableName = getUniqueNames(1)[0];
+      createTableAndDisableCompactions(accumuloClient, tableName);
+      final TableId tableId =
+          TableId.of(accumuloClient.tableOperations().tableIdMap().get(tableName));
+
+      // Ingest 10000 rows start with row 1 and flush and verify data
+      ingest(accumuloClient, rows, COLS, 10, 1, tableName);
+      accumuloClient.tableOperations().flush(tableName, null, null, true);
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+
+      // Bring tablet offline so we can modify file metadata
+      accumuloClient.tableOperations().offline(tableName, true);
+
+      try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+          .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) {
+
+        // Read each file (should only be 1), and split into 4 ranges
+        for (TabletMetadata tabletMetadata : tabletsMetadata) {
+          final KeyExtent ke = tabletMetadata.getExtent();
+
+          // Create a mutation to delete the existing file metadata entry with infinite range
+          TabletMutator mutator = ctx.getAmple().mutateTablet(ke);
+
+          // Read each of the 10 files referenced by the table, should be 1 per tablet
+          for (Entry<StoredTabletFile,DataFileValue> fileEntry : tabletMetadata.getFilesMap()
+              .entrySet()) {
+            StoredTabletFile file = fileEntry.getKey();
+            DataFileValue value = fileEntry.getValue();
+
+            // Create a mutation to delete the existing file metadata entry with infinite range
+            mutator.deleteFile(file);
+
+            // Add 4 contiguous ranges
+            final int ranges = 4;
+            for (int i = 1; i <= ranges; i++) {
+              int rowsPerRange = rows / ranges;
+              int endRow = i * rowsPerRange;
+
+              mutator.putFile(
+                  StoredTabletFile.of(file.getPath(),
+                      new Range(new Text("row_" + String.format("%010d", endRow - rowsPerRange)),
+                          false, new Text("row_" + String.format("%010d", endRow)), true)),
+                  new DataFileValue(value.getSize() / ranges, value.getNumEntries() / ranges));
+            }
+            mutator.mutate();
+          }
+        }
+      }
+
+      accumuloClient.tableOperations().online(tableName, true);
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+      // Should have 4 files
+      printAndVerifyFileMetadata(getServerContext(), tableId, 4);
+
+      // Compact and verify the correct rows are still valid
+      accumuloClient.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+      printAndVerifyFileMetadata(getServerContext(), tableId, 1);
+    }
+  }
+
+  @Test
+  public void fencedRangeTest() throws Exception {
+    ServerContext ctx = getCluster().getServerContext();
+
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      // Need permission to write to metadata
+      accumuloClient.securityOperations().grantTablePermission(accumuloClient.whoami(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+
+      final int rows = 10000;
+      final int ranges = 4;
+      int rowsPerRange = rows / ranges;
+
+      final String tableName = getUniqueNames(1)[0];
+      createTableAndDisableCompactions(accumuloClient, tableName);
+      final TableId tableId =
+          TableId.of(accumuloClient.tableOperations().tableIdMap().get(tableName));
+
+      // Ingest 10000 rows start with row 1 and flush and verify data
+      ingest(accumuloClient, rows, COLS, 10, 1, tableName);
+      accumuloClient.tableOperations().flush(tableName, null, null, true);
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+
+      // Bring tablet offline so we can modify file metadata
+      accumuloClient.tableOperations().offline(tableName, true);
+
+      try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+          .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) {
+
+        // Read each file, should just be 1.
+        // Split into 4 ranges and skip the second so only 3/4 of the file should be readable
+        for (TabletMetadata tabletMetadata : tabletsMetadata) {
+          var fileEntry = tabletMetadata.getFilesMap().entrySet().stream()
+              .collect(MoreCollectors.onlyElement());
+          StoredTabletFile file = fileEntry.getKey();
+          DataFileValue value = fileEntry.getValue();
+          final KeyExtent ke = tabletMetadata.getExtent();
+
+          // Create a mutation to delete the existing file metadata entry with infinite range
+          TabletMutator mutator = ctx.getAmple().mutateTablet(ke);
+          mutator.deleteFile(file);
+
+          // Add 3 ranges
+          for (int i = 1; i <= ranges; i++) {
+            // Skip second range
+            if (i == 2) {
+              continue;
+            }
+            int endRow = i * rowsPerRange;
+            mutator.putFile(
+                StoredTabletFile.of(file.getPath(),
+                    new Range(new Text("row_" + String.format("%010d", endRow - rowsPerRange)),
+                        false, new Text("row_" + String.format("%010d", endRow)), true)),
+                new DataFileValue(value.getSize() / ranges, value.getNumEntries() / ranges));
+          }
+
+          mutator.mutate();
+        }
+      }
+
+      // Write mutations to metadata and then bring the table back online
+      accumuloClient.tableOperations().online(tableName, true);
+
+      // Verify rows 1 - 2500 are readable
+      verify(accumuloClient, rowsPerRange, COLS, 10, 1, tableName);
+      // Rows 2501 - 5000 should not be fenced and not visible
+      // Try and read 2500 rows and verify none are visible, should throw an exception with 0 rows
+      // read
+      verifyNoRows(accumuloClient, rowsPerRange, COLS, 10, rowsPerRange + 1, tableName);
+      // Verify rows 5001 - 10000 are readable
+      verify(accumuloClient, rowsPerRange * 2, COLS, 10, (rowsPerRange * 2) + 1, tableName);
+
+      // Should have 3 rfiles in metadata
+      printAndVerifyFileMetadata(getServerContext(), tableId, 3);
+      // Compact and verify the correct rows are still valid
+      accumuloClient.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+
+      // Verify rows 1 - 2500 are readable
+      verify(accumuloClient, rowsPerRange, COLS, 10, 1, tableName);
+      // Rows 2501 - 5000 should not be fenced and not visible
+      // Try and read 2500 rows and verify none are visible, should throw an exception with 0
+      // rows read
+      verifyNoRows(accumuloClient, rowsPerRange, COLS, 10, rowsPerRange + 1, tableName);
+      // Verify rows 5001 - 10000 are readable
+      verify(accumuloClient, rowsPerRange * 2, COLS, 10, (rowsPerRange * 2) + 1, tableName);
+      // Should just have 1 file after compaction
+      printAndVerifyFileMetadata(getServerContext(), tableId, 1);
+
+    }
+  }
+
+  @Test
+  public void splitsRangeTest() throws Exception {
+    ServerContext ctx = getCluster().getServerContext();
+
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      // Need permission to write to metadata
+      accumuloClient.securityOperations().grantTablePermission(accumuloClient.whoami(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+
+      final int rows = 100000;
+      final String tableName = getUniqueNames(1)[0];
+      createTableAndDisableCompactions(accumuloClient, tableName);
+      final TableId tableId =
+          TableId.of(accumuloClient.tableOperations().tableIdMap().get(tableName));
+
+      // Divide table into 10 tablets with end rows of 10000, 20000, etc.
+      final SortedSet<Text> splits = new TreeSet<>();
+      for (int i = 1; i <= 10; i++) {
+        splits.add(new Text("row_" + String.format("%010d", ((i * 10000)))));
+      }
+
+      // Ingest 100000 rows start with row 1 and flush and verify data
+      accumuloClient.tableOperations().addSplits(tableName, splits);
+      ingest(accumuloClient, rows, COLS, 10, 1, tableName);
+      accumuloClient.tableOperations().flush(tableName, null, null, true);
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+
+      // Bring tablet offline so we can modify file metadata
+      accumuloClient.tableOperations().offline(tableName, true);
+
+      try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+          .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) {
+
+        // Read each file, should be 10
+        // Split into 2 ranges
+        for (TabletMetadata tabletMetadata : tabletsMetadata) {
+          final KeyExtent ke = tabletMetadata.getExtent();
+
+          // Create a mutation to delete the existing file metadata entry with infinite range
+          TabletMutator mutator = ctx.getAmple().mutateTablet(ke);
+
+          // Read each of the 10 files referenced by the table, should be 1 per tablet
+          for (Entry<StoredTabletFile,DataFileValue> fileEntry : tabletMetadata.getFilesMap()
+              .entrySet()) {
+            StoredTabletFile file = fileEntry.getKey();
+            DataFileValue value = fileEntry.getValue();
+            final int endRow = Integer.parseInt(ke.endRow().toString().replace("row_", ""));
+
+            mutator.deleteFile(file);
+            mutator.putFile(
+                StoredTabletFile.of(file.getPath(),
+                    new Range(new Text("row_" + String.format("%010d", (endRow - 10000))), false,
+                        new Text("row_" + String.format("%010d", (endRow - 5000))), true)),
+                new DataFileValue(value.getSize() / 2, value.getNumEntries() / 2));
+            mutator.putFile(
+                StoredTabletFile.of(file.getPath(),
+                    new Range(new Text("row_" + String.format("%010d", (endRow - 5000))), false,
+                        new Text("row_" + String.format("%010d", endRow)), true)),
+                new DataFileValue(value.getSize() / 2, value.getNumEntries() / 2));
+
+            mutator.mutate();
+          }
+        }
+      }
+
+      accumuloClient.tableOperations().online(tableName, true);
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+
+      printAndVerifyFileMetadata(getServerContext(), tableId, 20);
+      // Compact and verify the correct rows are still valid
+      accumuloClient.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+      printAndVerifyFileMetadata(getServerContext(), tableId, 10);
+
+    }
+  }
+
+  @Test
+  public void splitsWithExistingRangesTest() throws Exception {
+    ServerContext ctx = getCluster().getServerContext();
+
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      // Need permission to write to metadata
+      accumuloClient.securityOperations().grantTablePermission(accumuloClient.whoami(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+
+      final int rows = 100000;
+      final int ranges = 4;
+      int rowsPerRange = rows / ranges;
+
+      final String tableName = getUniqueNames(1)[0];
+      createTableAndDisableCompactions(accumuloClient, tableName);
+      final TableId tableId =
+          TableId.of(accumuloClient.tableOperations().tableIdMap().get(tableName));
+
+      final SortedSet<Text> splits = new TreeSet<>();
+
+      // Ingest 10000 rows start with row 1 and flush and verify data
+      ingest(accumuloClient, rows, COLS, 10, 1, tableName);
+      accumuloClient.tableOperations().flush(tableName, null, null, true);
+      verify(accumuloClient, rows, COLS, 10, 1, tableName);
+
+      // Bring tablet offline so we can modify file metadata
+      accumuloClient.tableOperations().offline(tableName, true);
+
+      try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+          .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) {
+
+        // Read each file, should just be 1.
+        // Split into 4 ranges and skip the second so only 3/4 of the file should be readable
+        for (TabletMetadata tabletMetadata : tabletsMetadata) {
+          var fileEntry = tabletMetadata.getFilesMap().entrySet().stream()
+              .collect(MoreCollectors.onlyElement());
+          StoredTabletFile file = fileEntry.getKey();
+          DataFileValue value = fileEntry.getValue();
+          final KeyExtent ke = tabletMetadata.getExtent();
+
+          // Create a mutation to delete the existing file metadata entry with infinite range
+          TabletMutator mutator = ctx.getAmple().mutateTablet(ke);
+          mutator.deleteFile(file);
+
+          // Add 3 ranges
+          for (int i = 1; i <= ranges; i++) {
+            // Skip second range
+            if (i == 2) {
+              continue;
+            }
+            int endRow = i * rowsPerRange;
+            mutator.putFile(
+                StoredTabletFile.of(file.getPath(),
+                    new Range(new Text("row_" + String.format("%010d", endRow - rowsPerRange)),
+                        false, new Text("row_" + String.format("%010d", endRow)), true)),
+                new DataFileValue(value.getSize() / ranges, value.getNumEntries() / ranges));
+          }
+
+          mutator.mutate();
+        }
+      }
+      accumuloClient.tableOperations().online(tableName, true);
+      printAndVerifyFileMetadata(getServerContext(), tableId, 3);
+      // Verify rows 1 - 2500 are readable
+      verify(accumuloClient, rowsPerRange, COLS, 10, 1, tableName);
+      // Rows 2501 - 5000 should not be fenced and not visible
+      // Try and read 2500 rows and verify none are visible, should throw an exception
+      // with 0 rows read
+      verifyNoRows(accumuloClient, rowsPerRange, COLS, 10, rowsPerRange + 1, tableName);
+      // Verify rows 5001 - 10000 are readable
+      verify(accumuloClient, rowsPerRange * 2, COLS, 10, (rowsPerRange * 2) + 1, tableName);
+
+      // Divide table into 10 tablets with end rows of 10000, 20000, etc.
+      splits.clear();
+      for (int i = 1; i <= 10; i++) {
+        splits.add(new Text("row_" + String.format("%010d", ((i * 10000)))));
+      }
+
+      // Ingest 100000 rows start with row 1 and flush and verify data
+      accumuloClient.tableOperations().addSplits(tableName, splits);
+      accumuloClient.tableOperations().flush(tableName, null, null, true);
+      // Verify rows 1 - 2500 are readable
+      verify(accumuloClient, rowsPerRange, COLS, 10, 1, tableName);
+      // Rows 2501 - 5000 should not be fenced and not visible
+      // Try and read 2500 rows and verify none are visible, should throw an exception
+      // with 0 rows read
+      verifyNoRows(accumuloClient, rowsPerRange, COLS, 10, rowsPerRange + 1, tableName);
+      // Verify rows 5001 - 10000 are readable
+      verify(accumuloClient, rowsPerRange * 2, COLS, 10, (rowsPerRange * 2) + 1, tableName);
+
+      // There are 3 existing ranges of 25000, and with splits of 10000 rows we end up
+      // with overlap and should be 9 total files for the existing data
+      // range: 1 - 25000; splits: 10000, 20000, 30000
+      // range: 50001 - 75000; splits: 60000, 70000, 80000
+      // range: 75001 - 100000; splits: 80000, 90000, 100000
+      printAndVerifyFileMetadata(getServerContext(), tableId, 9);
+
+      // // Compact and verify the correct rows are still valid
+      accumuloClient.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+      // Verify rows 1 - 2500 are readable
+      verify(accumuloClient, rowsPerRange, COLS, 10, 1, tableName);
+      // Rows 2501 - 5000 should not be fenced and not visible
+      // Try and read 2500 rows and verify none are visible, should throw an exception
+      // with 0 rows read
+      verifyNoRows(accumuloClient, rowsPerRange, COLS, 10, rowsPerRange + 1, tableName);
+      // Verify rows 5001 - 10000 are readable
+      verify(accumuloClient, rowsPerRange * 2, COLS, 10, (rowsPerRange * 2) + 1, tableName);
+
+      // After compaction should be 8 files because data was ingested into
+      // 1 - 25000, 50001 - 10000. 25001 - 50000 was skipped so splits of
+      // 40000 and 50000 do not have data or files.
+      printAndVerifyFileMetadata(getServerContext(), tableId, 8);
+    }
+  }
+
+  // In the future we should probably enhance the ingest verify code to be able to better verify
+  // ranges
+  // but for now we can at least verify no rows are read by checking the exception
+  private static void verifyNoRows(AccumuloClient accumuloClient, int rows, int cols, int width,
+      int offset, String tableName) throws Exception {
+    try {
+      verify(accumuloClient, rows, cols, width, offset, tableName);
+      fail("Should have failed");
+    } catch (AccumuloException e) {
+      assertTrue(e.getMessage().contains("Did not read expected number of rows. Saw 0"));
+    }
+  }
+
+  private static void createTableAndDisableCompactions(AccumuloClient c, String tableName)
+      throws Exception {
+    // disable compactions
+    NewTableConfiguration ntc = new NewTableConfiguration();
+    ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "9999"));
+    c.tableOperations().create(tableName, ntc);
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FileNormalizationIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FileNormalizationIT.java
index 25a54c9..4ce9c84 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FileNormalizationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FileNormalizationIT.java
@@ -36,13 +36,13 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.VerifyIngest;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
@@ -97,7 +97,7 @@
         scanner.forEach((k, v) -> {
           var row = k.getRowData().toString();
           var qual = k.getColumnQualifierData().toString();
-          var path = new Path(qual).toString();
+          var path = StoredTabletFile.of(k.getColumnQualifierData().toString()).getMetadataPath();
           var rowPath = row + "+" + path;
 
           log.debug("split test, inspecting {} {} {}", row, qual, v);
@@ -132,7 +132,7 @@
         Set<String> filenames = new HashSet<>();
 
         scanner.forEach((k, v) -> {
-          var path = new Path(k.getColumnQualifierData().toString());
+          var path = StoredTabletFile.of(k.getColumnQualifierData().toString()).getPath();
           assertFalse(filenames.contains(path.getName()));
           assertTrue(path.getName().startsWith("A"));
           filenames.add(path.getName());
@@ -174,7 +174,7 @@
         scanner.forEach((k, v) -> {
           var qual = k.getColumnQualifierData().toString();
           assertTrue(qual.contains("//tables//"));
-          filesBeforeMerge.add(qual);
+          filesBeforeMerge.add(StoredTabletFile.of(qual).getMetadataPath());
         });
       }
 
@@ -188,7 +188,7 @@
         scanner.forEach((k, v) -> {
           // should only see the default tablet
           assertTrue(k.getRow().toString().endsWith("<"));
-          filesAfterMerge.add(k.getColumnQualifierData().toString());
+          filesAfterMerge.add(StoredTabletFile.of(k.getColumnQualifier()).getMetadataPath());
         });
       }
 
@@ -200,7 +200,7 @@
   }
 
   private Scanner createMetadataFileScanner(AccumuloClient client, String table) throws Exception {
-    var scanner = client.createScanner(MetadataTable.NAME);
+    var scanner = client.createScanner(AccumuloTable.METADATA.tableName());
     var tableId = TableId.of(client.tableOperations().tableIdMap().get(table));
     var range = new KeyExtent(tableId, null, null).toMetaRange();
     scanner.setRange(range);
@@ -211,11 +211,11 @@
   private void misnormalizeFiles(AccumuloClient client, String table) throws Exception {
     client.tableOperations().offline(table, true);
 
-    client.securityOperations().grantTablePermission(getPrincipal(), MetadataTable.NAME,
-        TablePermission.WRITE);
+    client.securityOperations().grantTablePermission(getPrincipal(),
+        AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
 
     try (var scanner = createMetadataFileScanner(client, table);
-        var writer = client.createBatchWriter(MetadataTable.NAME)) {
+        var writer = client.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       scanner.forEach((k, v) -> {
         Mutation m = new Mutation(k.getRow());
         var qual = k.getColumnQualifierData().toString();
@@ -230,8 +230,8 @@
         }
       });
     } finally {
-      client.securityOperations().revokeTablePermission(getPrincipal(), MetadataTable.NAME,
-          TablePermission.WRITE);
+      client.securityOperations().revokeTablePermission(getPrincipal(),
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
       client.tableOperations().online(table, true);
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
index 6d93b95..0b099b0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/FunctionalTestUtils.java
@@ -44,6 +44,7 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.cluster.AccumuloCluster;
 import org.apache.accumulo.core.Constants;
@@ -58,9 +59,10 @@
 import org.apache.accumulo.core.fate.AdminUtil;
 import org.apache.accumulo.core.fate.AdminUtil.FateStatus;
 import org.apache.accumulo.core.fate.ZooStore;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -80,7 +82,8 @@
 public class FunctionalTestUtils {
 
   public static int countRFiles(AccumuloClient c, String tableName) throws Exception {
-    try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName));
       scanner.setRange(TabletsSection.getRange(tableId));
       scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
@@ -89,21 +92,27 @@
   }
 
   public static List<String> getRFilePaths(AccumuloClient c, String tableName) throws Exception {
-    List<String> files = new ArrayList<>();
-    try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    return getStoredTabletFiles(c, tableName).stream().map(StoredTabletFile::getMetadataPath)
+        .collect(Collectors.toList());
+  }
+
+  public static List<StoredTabletFile> getStoredTabletFiles(AccumuloClient c, String tableName)
+      throws Exception {
+    List<StoredTabletFile> files = new ArrayList<>();
+    try (Scanner scanner =
+        c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName));
       scanner.setRange(TabletsSection.getRange(tableId));
       scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      scanner.forEach(entry -> {
-        files.add(entry.getKey().getColumnQualifier().toString());
-      });
+      scanner.forEach(entry -> files.add(StoredTabletFile.of(entry.getKey().getColumnQualifier())));
     }
     return files;
   }
 
   static void checkRFiles(AccumuloClient c, String tableName, int minTablets, int maxTablets,
       int minRFiles, int maxRFiles) throws Exception {
-    try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner scanner =
+        c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       String tableId = c.tableOperations().tableIdMap().get(tableName);
       scanner.setRange(new Range(new Text(tableId + ";"), true, new Text(tableId + "<"), true));
       scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
@@ -127,14 +136,17 @@
       }
 
       if (tabletFileCounts.size() < minTablets || tabletFileCounts.size() > maxTablets) {
-        throw new Exception("Did not find expected number of tablets " + tabletFileCounts.size());
+        throw new Exception("table " + tableName + " has unexpected number of tablets. Found: "
+            + tabletFileCounts.size() + ". expected " + minTablets + " < numTablets < "
+            + maxTablets);
       }
 
       Set<Entry<Text,Integer>> es = tabletFileCounts.entrySet();
       for (Entry<Text,Integer> entry : es) {
         if (entry.getValue() > maxRFiles || entry.getValue() < minRFiles) {
           throw new Exception(
-              "tablet " + entry.getKey() + " has " + entry.getValue() + " map files");
+              "tablet " + entry.getKey() + " has unexpected number of data files. Found: "
+                  + entry.getValue() + ". expected " + minTablets + " < numFiles < " + maxTablets);
         }
       }
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index 1776bc4..bf82331 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -27,14 +27,14 @@
 import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.UncheckedIOException;
+import java.net.URI;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
+import java.util.Optional;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
@@ -48,12 +48,14 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 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.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
@@ -61,8 +63,6 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection.SkewedKeyValue;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
@@ -84,6 +84,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterators;
+import com.google.common.net.HostAndPort;
 
 public class GarbageCollectorIT extends ConfigurableMacBase {
   private static final String OUR_SECRET = "itsreallysecret";
@@ -147,7 +148,7 @@
       log.info("Counted {} files in path: {}", before, pathString);
 
       while (true) {
-        sleepUninterruptibly(1, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(1));
         int more = countFiles(pathString);
         if (more <= before) {
           break;
@@ -158,7 +159,7 @@
       // restart GC
       log.info("Restarting GC...");
       getCluster().start();
-      sleepUninterruptibly(15, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(15));
       log.info("Again Counting files in path: {}", pathString);
 
       int after = countFiles(pathString);
@@ -178,7 +179,7 @@
       addEntries(c);
       cluster.getConfig().setDefaultMemory(32, MemoryUnit.MEGABYTE);
       ProcessInfo gc = cluster.exec(SimpleGarbageCollector.class);
-      sleepUninterruptibly(20, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(20));
       String output = "";
       while (!output.contains("has exceeded the threshold")) {
         try {
@@ -202,7 +203,7 @@
       c.tableOperations().create(table);
       // let gc run for a bit
       cluster.start();
-      sleepUninterruptibly(20, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(20));
       killMacGc();
       // kill tservers
       for (ProcessReference ref : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
@@ -211,7 +212,8 @@
       // run recovery
       cluster.start();
       // did it recover?
-      try (Scanner scanner = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner scanner =
+          c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         scanner.forEach((k, v) -> {});
       }
     }
@@ -240,9 +242,9 @@
       c.tableOperations().flush(table, null, null, true);
 
       // ensure an invalid delete entry does not cause GC to go berserk ACCUMULO-2520
-      c.securityOperations().grantTablePermission(c.whoami(), MetadataTable.NAME,
+      c.securityOperations().grantTablePermission(c.whoami(), AccumuloTable.METADATA.tableName(),
           TablePermission.WRITE);
-      try (BatchWriter bw = c.createBatchWriter(MetadataTable.NAME)) {
+      try (BatchWriter bw = c.createBatchWriter(AccumuloTable.METADATA.tableName())) {
         bw.addMutation(createDelMutation("", "", "", ""));
         bw.addMutation(createDelMutation("", "testDel", "test", "valueTest"));
         // path is invalid but value is expected - only way the invalid entry will come through
@@ -255,7 +257,7 @@
       try {
         String output = "";
         while (!output.contains("Ignoring invalid deletion candidate")) {
-          sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
+          Thread.sleep(250);
           try {
             output = gc.readStdOut();
           } catch (UncheckedIOException ioe) {
@@ -341,8 +343,8 @@
         List.of(new GcCandidate("hdfs://foo.com:6000/user/foo/tables/+r/t-0/F00.rf", 0L),
             new GcCandidate("hdfs://foo.com:6000/user/foo/tables/+r/t-0/F001.rf", 1L));
 
-    List<StoredTabletFile> stfs = new LinkedList<>();
-    candidates.stream().forEach(temp -> stfs.add(new StoredTabletFile(temp.getPath())));
+    List<StoredTabletFile> stfs = candidates.stream()
+        .map(temp -> StoredTabletFile.of(new Path(temp.getPath()))).collect(Collectors.toList());
 
     log.debug("Adding root table GcCandidates");
     ample.putGcCandidates(tableId, stfs);
@@ -353,7 +355,8 @@
     int counter = 0;
     while (cIter.hasNext()) {
       // Duplicate these entries back into zookeeper
-      ample.putGcCandidates(tableId, List.of(new StoredTabletFile(cIter.next().getPath())));
+      ample.putGcCandidates(tableId,
+          List.of(StoredTabletFile.of(new Path(cIter.next().getPath()))));
       counter++;
     }
     // Ensure Zookeeper collapsed the entries and did not support duplicates.
@@ -422,22 +425,15 @@
         if (locks != null && !locks.isEmpty()) {
           String lockPath = path + "/" + locks.get(0);
 
-          String gcLoc = new String(zk.getData(lockPath));
+          Optional<ServiceLockData> sld = ServiceLockData.parse(zk.getData(lockPath));
 
-          assertTrue(gcLoc.startsWith(Service.GC_CLIENT.name()),
-              "Found unexpected data in zookeeper for GC location: " + gcLoc);
-          int loc = gcLoc.indexOf(ServerServices.SEPARATOR_CHAR);
-          assertNotEquals(-1, loc, "Could not find split point of GC location for: " + gcLoc);
-          String addr = gcLoc.substring(loc + 1);
+          assertNotNull(sld.orElseThrow());
+          HostAndPort hostAndPort = sld.orElseThrow().getAddress(ThriftService.GC);
 
-          int addrSplit = addr.indexOf(':');
-          assertNotEquals(-1, addrSplit, "Could not find split of GC host:port for: " + addr);
-
-          String host = addr.substring(0, addrSplit), port = addr.substring(addrSplit + 1);
           // We shouldn't have the "bindall" address in zk
-          assertNotEquals("0.0.0.0", host);
+          assertNotEquals("0.0.0.0", hostAndPort.getHost());
           // Nor should we have the "random port" in zk
-          assertNotEquals(0, Integer.parseInt(port));
+          assertNotEquals(0, hostAndPort.getPort());
           return;
         }
 
@@ -455,14 +451,15 @@
 
   private void addEntries(AccumuloClient client) throws Exception {
     Ample ample = getServerContext().getAmple();
-    client.securityOperations().grantTablePermission(client.whoami(), MetadataTable.NAME,
-        TablePermission.WRITE);
-    try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
+    client.securityOperations().grantTablePermission(client.whoami(),
+        AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+    try (BatchWriter bw = client.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       for (int i = 0; i < 100000; ++i) {
         String longpath = "aaaaaaaaaabbbbbbbbbbccccccccccddddddddddeeeeeeeeee"
             + "ffffffffffgggggggggghhhhhhhhhhiiiiiiiiiijjjjjjjjjj";
-        var path = String.format("file:/%020d/%s", i, longpath);
-        Mutation delFlag = ample.createDeleteMutation(ReferenceFile.forFile(TableId.of("1"), path));
+        var path = URI.create(String.format("file:/%020d/%s", i, longpath));
+        Mutation delFlag =
+            ample.createDeleteMutation(ReferenceFile.forFile(TableId.of("1"), new Path(path)));
         bw.addMutation(delFlag);
       }
     }
@@ -492,9 +489,9 @@
 
     // Create multiple candidate entries
     List<StoredTabletFile> stfs = Stream
-        .of(new StoredTabletFile("hdfs://foo.com:6000/user/foo/tables/a/t-0/F00.rf"),
-            new StoredTabletFile("hdfs://foo.com:6000/user/foo/tables/b/t-0/F00.rf"))
-        .collect(Collectors.toList());
+        .of("hdfs://foo.com:6000/user/foo/tables/a/t-0/F00.rf",
+            "hdfs://foo.com:6000/user/foo/tables/b/t-0/F00.rf")
+        .map(Path::new).map(StoredTabletFile::of).collect(Collectors.toList());
 
     log.debug("Adding candidates to table {}", tableId);
     ample.putGcCandidates(tableId, stfs);
@@ -510,7 +507,8 @@
 
     GcCandidate deleteCandidate = candidates.get(0);
     assertNotNull(deleteCandidate);
-    ample.putGcCandidates(tableId, List.of(new StoredTabletFile(deleteCandidate.getPath())));
+    ample.putGcCandidates(tableId,
+        List.of(StoredTabletFile.of(new Path(deleteCandidate.getPath()))));
 
     log.debug("Deleting Candidate {}", deleteCandidate);
     ample.deleteGcCandidates(datalevel, List.of(deleteCandidate), type);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java
index ac9c0b9..8904dad 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java
@@ -26,11 +26,12 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.data.TableId;
+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.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.server.ServerContext;
@@ -51,14 +52,14 @@
   protected ArrayList<StoredTabletFile> getFilesForTable(ServerContext ctx, AccumuloClient client,
       String tableName) {
     String tid = client.tableOperations().tableIdMap().get(tableName);
-    TabletsMetadata tms =
-        ctx.getAmple().readTablets().forTable(TableId.of(tid)).fetch(ColumnType.FILES).build();
-    ArrayList<StoredTabletFile> files = new ArrayList<>();
-    tms.forEach(tm -> {
-      files.addAll(tm.getFiles());
-    });
-    LOG.debug("Tablet files: {}", files);
-    return files;
+    try (TabletsMetadata tms =
+        ctx.getAmple().readTablets().forTable(TableId.of(tid)).fetch(ColumnType.FILES).build()) {
+      ArrayList<StoredTabletFile> files =
+          tms.stream().flatMap(tabletMetadata -> tabletMetadata.getFiles().stream())
+              .collect(Collectors.toCollection(ArrayList::new));
+      LOG.debug("Tablet files: {}", files);
+      return files;
+    }
   }
 
   protected ArrayList<StoredTabletFile> loadData(ServerContext ctx, AccumuloClient client,
@@ -112,7 +113,7 @@
         if (lfs.isDirectory()) {
           continue;
         }
-        TabletFile tf = new TabletFile(lfs.getPath());
+        ReferencedTabletFile tf = new ReferencedTabletFile(lfs.getPath());
         LOG.debug("File in trash: {}, tableId: {}", lfs.getPath(), tf.getTableId());
         if (tid.equals(tf.getTableId())) {
           count++;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
index b8faff9..d98a7b7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
@@ -38,8 +38,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.junit.jupiter.api.Test;
 
-// verify trash is not used with Hadoop default configuration as Trash is not
-// enabled by default.
+// verify trash is not used with Hadoop defaults, since Trash is not enabled by default
 public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase {
 
   @Override
@@ -49,7 +48,7 @@
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    // By default Hadoop trash is disabled - fs.trash.interval defaults to 0
+    // By default Hadoop trash is disabled - fs.trash.interval defaults to 0; ensure it's 0 here
     Map<String,String> hadoopOverrides = new HashMap<>();
     hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "0");
     cfg.setHadoopConfOverrides(hadoopOverrides);
@@ -57,9 +56,6 @@
 
     cfg.setProperty(Property.GC_CYCLE_START, "1");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
-    @SuppressWarnings("removal")
-    Property p = Property.GC_TRASH_IGNORE;
-    cfg.setProperty(p, "false"); // default, use trash if configured
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
@@ -76,13 +72,9 @@
       assertFalse(files.isEmpty());
       c.tableOperations().compact(table, new CompactionConfig());
       TableId tid = TableId.of(c.tableOperations().tableIdMap().get(table));
-      // The default value for fs.trash.interval is 0, which means that
-      // trash is disabled in the Hadoop configuration. Enabling trash in
-      // Accumulo (GC_TRASH_IGNORE = false) still requires enabling trash in Hadoop
       super.waitForFilesToBeGCd(files);
       assertEquals(0, super.countFilesInTrash(fs, tid));
     }
-
   }
 
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.java
deleted file mode 100644
index d1985fa..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.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.test.functional;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.admin.CompactionConfig;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FileSystem;
-import org.junit.jupiter.api.Test;
-
-// verify trash is not used when Hadoop is configured to enable it and our property
-// is set to ignore it and delete the file anyway
-public class GarbageCollectorTrashDisabledIT extends GarbageCollectorTrashBase {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(5);
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-
-    Map<String,String> hadoopOverrides = new HashMap<>();
-    hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "5");
-    cfg.setHadoopConfOverrides(hadoopOverrides);
-    cfg.useMiniDFS(true);
-
-    cfg.setProperty(Property.GC_CYCLE_START, "1");
-    cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
-    @SuppressWarnings("removal")
-    Property p = Property.GC_TRASH_IGNORE;
-    cfg.setProperty(p, "true"); // don't use trash if configured
-    cfg.setProperty(Property.GC_PORT, "0");
-    cfg.setProperty(Property.TSERV_MAXMEM, "5K");
-    cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
-    cfg.setProperty(Property.TSERV_MAJC_DELAY, "180s");
-  }
-
-  @Test
-  public void testTrashHadoopEnabledAccumuloDisabled() throws Exception {
-    String table = this.getUniqueNames(1)[0];
-    final FileSystem fs = super.getCluster().getFileSystem();
-    super.makeTrashDir(fs);
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-      ArrayList<StoredTabletFile> files = super.loadData(super.getServerContext(), c, table);
-      assertFalse(files.isEmpty());
-      c.tableOperations().compact(table, new CompactionConfig());
-      TableId tid = TableId.of(c.tableOperations().tableIdMap().get(table));
-      super.waitForFilesToBeGCd(files);
-      // Trash is disabled in Accumulo (GC_TRASH_IGNORE = true)
-      // no files for this table should be in the trash
-      assertEquals(0, super.countFilesInTrash(fs, tid));
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
index 603d3f8..ca59797 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
@@ -38,8 +38,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.junit.jupiter.api.Test;
 
-// verify that trash is used if our property is set to not ignore it (the default)
-// and Hadoop is configured to enable it
+// verify that trash is used if Hadoop is configured to use it
 public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase {
 
   @Override
@@ -49,7 +48,7 @@
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-
+    // By default Hadoop trash is disabled - fs.trash.interval defaults to 0; override that here
     Map<String,String> hadoopOverrides = new HashMap<>();
     hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "5");
     cfg.setHadoopConfOverrides(hadoopOverrides);
@@ -57,9 +56,6 @@
 
     cfg.setProperty(Property.GC_CYCLE_START, "1");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
-    @SuppressWarnings("removal")
-    Property p = Property.GC_TRASH_IGNORE;
-    cfg.setProperty(p, "false"); // default, use trash if configured
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java
similarity index 91%
rename from test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java
rename to test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java
index 6331a71..9d5f06b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java
@@ -42,9 +42,8 @@
 import org.apache.hadoop.fs.TrashPolicyDefault;
 import org.junit.jupiter.api.Test;
 
-// verify that trash is used if our property is set to not ignore it (the default)
-// and Hadoop Trash is configured to enable it and use a custom policy.
-public class GarbageCollectorTrashEnabledCustomPolicyIT extends GarbageCollectorTrashBase {
+// verify that trash is used if Hadoop is configured to use it and that using a custom policy works
+public class GarbageCollectorTrashEnabledWithCustomPolicyIT extends GarbageCollectorTrashBase {
 
   public static class NoFlushFilesInTrashPolicy extends TrashPolicyDefault {
 
@@ -66,7 +65,7 @@
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-
+    // By default Hadoop trash is disabled - fs.trash.interval defaults to 0; override that here
     Map<String,String> hadoopOverrides = new HashMap<>();
     hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "5");
     hadoopOverrides.put("fs.trash.classname", NoFlushFilesInTrashPolicy.class.getName());
@@ -75,9 +74,6 @@
 
     cfg.setProperty(Property.GC_CYCLE_START, "1");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
-    @SuppressWarnings("removal")
-    Property p = Property.GC_TRASH_IGNORE;
-    cfg.setProperty(p, "false"); // default, use trash if configured
     cfg.setProperty(Property.GC_PORT, "0");
     cfg.setProperty(Property.TSERV_MAXMEM, "5K");
     cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfClosedTabletIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfClosedTabletIT.java
index 8f6d907..82a8e72 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/HalfClosedTabletIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfClosedTabletIT.java
@@ -40,7 +40,6 @@
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -52,6 +51,8 @@
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
@@ -63,6 +64,8 @@
 //
 public class HalfClosedTabletIT extends SharedMiniClusterBase {
 
+  public static final Logger log = LoggerFactory.getLogger(HalfClosedTabletIT.class);
+
   public static class HalfClosedTabletITConfiguration implements MiniClusterConfigurationCallback {
 
     @Override
@@ -115,7 +118,12 @@
       Thread.sleep(3000);
 
       Thread configFixer = new Thread(() -> {
-        UtilWaitThread.sleep(3000);
+        try {
+          Thread.sleep(3000);
+        } catch (InterruptedException ex) {
+          // ignore exception
+          Thread.currentThread().interrupt();
+        }
         removeInvalidClassLoaderContextProperty(client, tableName);
       });
 
@@ -214,7 +222,7 @@
 
       c.tableOperations().flush(tableName, null, null, false);
 
-      UtilWaitThread.sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(5000);
 
       // minc should fail, so there should be no files
       FunctionalTestUtils.checkRFiles(c, tableName, 1, 1, 0, 0);
@@ -225,7 +233,7 @@
       // The offine operation should not be able to complete because the tablet can not minor
       // compact, give the offline operation a bit of time to attempt to complete even though it
       // should never be able to complete.
-      UtilWaitThread.sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(5000);
 
       assertTrue(countHostedTablets(c, tid) > 0);
 
@@ -269,6 +277,7 @@
       FunctionalTestUtils.checkRFiles(c, tableName, minTablets, maxTablets, minRFiles, maxRFiles);
       return true;
     } catch (Exception e) {
+      log.info(e.getMessage());
       return false;
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
index 7a017c9..de85287 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/HalfDeadTServerIT.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
@@ -31,7 +31,6 @@
 import java.time.Duration;
 import java.util.Map;
 import java.util.Scanner;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -186,24 +185,24 @@
       try {
         stderrCollector.start();
         stdoutCollector.start();
-        sleepUninterruptibly(1, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(1));
         // don't need the regular tablet server
         cluster.killProcess(ServerType.TABLET_SERVER,
             cluster.getProcesses().get(ServerType.TABLET_SERVER).iterator().next());
-        sleepUninterruptibly(1, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(1));
         client.tableOperations().create("test_ingest");
         assertEquals(1, client.instanceOperations().getTabletServers().size());
         int rows = 100_000;
         ingest =
             cluster.exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--rows", rows + "")
                 .getProcess();
-        sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+        Thread.sleep(500);
 
         // block I/O with some side-channel trickiness
         File trickFile = new File(trickFilename);
         try {
           assertTrue(trickFile.createNewFile());
-          sleepUninterruptibly(seconds, TimeUnit.SECONDS);
+          Thread.sleep(SECONDS.toMillis(seconds));
         } finally {
           if (!trickFile.delete()) {
             log.error("Couldn't delete {}", trickFile);
@@ -216,7 +215,7 @@
           params.rows = rows;
           VerifyIngest.verifyIngest(client, params);
         } else {
-          sleepUninterruptibly(5, TimeUnit.SECONDS);
+          Thread.sleep(SECONDS.toMillis(5));
           tserver.waitFor();
           stderrCollector.join();
           stdoutCollector.join();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index 010c2b6..9e7a49b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -58,8 +58,7 @@
 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.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.SystemPermission;
@@ -176,7 +175,8 @@
       }
 
       // and the ability to modify the root and metadata tables
-      for (String table : Arrays.asList(RootTable.NAME, MetadataTable.NAME)) {
+      for (String table : Arrays.asList(AccumuloTable.ROOT.tableName(),
+          AccumuloTable.METADATA.tableName())) {
         assertTrue(client.securityOperations().hasTablePermission(client.whoami(), table,
             TablePermission.ALTER_TABLE));
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java b/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java
index bef85c4..3f9f7e8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/LargeRowIT.java
@@ -19,14 +19,13 @@
 package org.apache.accumulo.test.functional;
 
 import static java.util.Collections.singletonMap;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.time.Duration;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -126,7 +125,7 @@
               .setProperties(singletonMap(Property.TABLE_MAX_END_ROW_SIZE.getKey(), "256K"))
               .withSplits(splitPoints));
 
-      sleepUninterruptibly(3, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(3));
       test1(c);
       test2(c);
     }
@@ -139,7 +138,7 @@
     c.tableOperations().setProperty(REG_TABLE_NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(),
         "" + SPLIT_THRESH);
 
-    sleepUninterruptibly(timeoutFactor * 12, TimeUnit.SECONDS);
+    Thread.sleep(SECONDS.toMillis(timeoutFactor * 12));
     log.info("checking splits");
     FunctionalTestUtils.checkSplits(c, REG_TABLE_NAME, NUM_PRE_SPLITS / 2, NUM_PRE_SPLITS * 4);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/LocalityGroupIT.java b/test/src/main/java/org/apache/accumulo/test/functional/LocalityGroupIT.java
new file mode 100644
index 0000000..d0f07d9
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/LocalityGroupIT.java
@@ -0,0 +1,218 @@
+/*
+ * 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.test.functional;
+
+import static org.apache.accumulo.test.functional.ReadWriteIT.ROWS;
+import static org.apache.accumulo.test.functional.ReadWriteIT.ingest;
+import static org.apache.accumulo.test.functional.ReadWriteIT.m;
+import static org.apache.accumulo.test.functional.ReadWriteIT.t;
+import static org.apache.accumulo.test.functional.ReadWriteIT.verify;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.accumulo.cluster.standalone.StandaloneAccumuloCluster;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+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.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.admin.TableOperations;
+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.file.rfile.PrintInfo;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LocalityGroupIT extends AccumuloClusterHarness {
+
+  private static final Logger log = LoggerFactory.getLogger(LocalityGroupIT.class);
+
+  @Test
+  public void localityGroupPerf() throws Exception {
+    // verify that locality groups can make look-ups faster
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      final String tableName = getUniqueNames(1)[0];
+      accumuloClient.tableOperations().create(tableName);
+      accumuloClient.tableOperations().setProperty(tableName, "table.group.g1", "colf");
+      accumuloClient.tableOperations().setProperty(tableName, "table.groups.enabled", "g1");
+      ingest(accumuloClient, 2000, 1, 50, 0, tableName);
+      accumuloClient.tableOperations().compact(tableName, null, null, true, true);
+      try (BatchWriter bw = accumuloClient.createBatchWriter(tableName)) {
+        bw.addMutation(m("zzzzzzzzzzz", "colf2", "cq", "value"));
+      }
+      long now = System.currentTimeMillis();
+      try (Scanner scanner = accumuloClient.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.fetchColumnFamily(new Text("colf"));
+        scanner.forEach((k, v) -> {});
+      }
+      long diff = System.currentTimeMillis() - now;
+      now = System.currentTimeMillis();
+
+      try (Scanner scanner = accumuloClient.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.fetchColumnFamily(new Text("colf2"));
+        scanner.forEach((k, v) -> {});
+      }
+      long diff2 = System.currentTimeMillis() - now;
+      assertTrue(diff2 < diff);
+    }
+  }
+
+  /**
+   * create a locality group, write to it and ensure it exists in the RFiles that result
+   */
+  @Test
+  public void sunnyLG() throws Exception {
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      final String tableName = getUniqueNames(1)[0];
+      accumuloClient.tableOperations().create(tableName);
+      Map<String,Set<Text>> groups = new TreeMap<>();
+      groups.put("g1", Collections.singleton(t("colf")));
+      accumuloClient.tableOperations().setLocalityGroups(tableName, groups);
+      verifyLocalityGroupsInRFile(accumuloClient, tableName);
+    }
+  }
+
+  /**
+   * Pretty much identical to sunnyLG, but verifies locality groups are created when configured in
+   * NewTableConfiguration prior to table creation.
+   */
+  @Test
+  public void sunnyLGUsingNewTableConfiguration() throws Exception {
+    // create a locality group, write to it and ensure it exists in the RFiles that result
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      final String tableName = getUniqueNames(1)[0];
+      NewTableConfiguration ntc = new NewTableConfiguration();
+      Map<String,Set<Text>> groups = new HashMap<>();
+      groups.put("g1", Collections.singleton(t("colf")));
+      ntc.setLocalityGroups(groups);
+      accumuloClient.tableOperations().create(tableName, ntc);
+      verifyLocalityGroupsInRFile(accumuloClient, tableName);
+    }
+  }
+
+  private void verifyLocalityGroupsInRFile(final AccumuloClient accumuloClient,
+      final String tableName) throws Exception {
+    ingest(accumuloClient, 2000, 1, 50, 0, tableName);
+    verify(accumuloClient, 2000, 1, 50, 0, tableName);
+    accumuloClient.tableOperations().flush(tableName, null, null, true);
+    try (BatchScanner bscanner = accumuloClient
+        .createBatchScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY, 1)) {
+      String tableId = accumuloClient.tableOperations().tableIdMap().get(tableName);
+      bscanner.setRanges(
+          Collections.singletonList(new Range(new Text(tableId + ";"), new Text(tableId + "<"))));
+      bscanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+      boolean foundFile = false;
+      for (Map.Entry<Key,Value> entry : bscanner) {
+        foundFile = true;
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        PrintStream oldOut = System.out;
+        try (PrintStream newOut = new PrintStream(baos)) {
+          System.setOut(newOut);
+          List<String> args = new ArrayList<>();
+          args.add(StoredTabletFile.of(entry.getKey().getColumnQualifier()).getMetadataPath());
+          args.add("--props");
+          args.add(getCluster().getAccumuloPropertiesPath());
+          if (getClusterType() == ClusterType.STANDALONE && saslEnabled()) {
+            args.add("--config");
+            StandaloneAccumuloCluster sac = (StandaloneAccumuloCluster) cluster;
+            String hadoopConfDir = sac.getHadoopConfDir();
+            args.add(new Path(hadoopConfDir, "core-site.xml").toString());
+            args.add(new Path(hadoopConfDir, "hdfs-site.xml").toString());
+          }
+          log.info("Invoking PrintInfo with {}", args);
+          PrintInfo.main(args.toArray(new String[args.size()]));
+          newOut.flush();
+          String stdout = baos.toString();
+          assertTrue(stdout.contains("Locality group           : g1"));
+          assertTrue(stdout.contains("families        : [colf]"));
+        } finally {
+          System.setOut(oldOut);
+        }
+      }
+      assertTrue(foundFile);
+    }
+  }
+
+  @Test
+  public void localityGroupChange() throws Exception {
+    // Make changes to locality groups and ensure nothing is lost
+    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
+      String table = getUniqueNames(1)[0];
+      TableOperations to = accumuloClient.tableOperations();
+      to.create(table);
+      String[] config = {"lg1:colf", null, "lg1:colf,xyz", "lg1:colf,xyz;lg2:c1,c2"};
+      int i = 0;
+      for (String cfg : config) {
+        to.setLocalityGroups(table, getGroups(cfg));
+        ingest(accumuloClient, ROWS * (i + 1), 1, 50, ROWS * i, table);
+        to.flush(table, null, null, true);
+        verify(accumuloClient, 0, 1, 50, ROWS * (i + 1), table);
+        i++;
+      }
+      to.delete(table);
+      to.create(table);
+      config = new String[] {"lg1:colf", null, "lg1:colf,xyz", "lg1:colf;lg2:colf",};
+      i = 1;
+      for (String cfg : config) {
+        ingest(accumuloClient, ROWS * i, 1, 50, 0, table);
+        ingest(accumuloClient, ROWS * i, 1, 50, 0, "xyz", table);
+        to.setLocalityGroups(table, getGroups(cfg));
+        to.flush(table, null, null, true);
+        verify(accumuloClient, ROWS * i, 1, 50, 0, table);
+        verify(accumuloClient, ROWS * i, 1, 50, 0, "xyz", table);
+        i++;
+      }
+    }
+  }
+
+  private Map<String,Set<Text>> getGroups(String cfg) {
+    Map<String,Set<Text>> groups = new TreeMap<>();
+    if (cfg != null) {
+      for (String group : cfg.split(";")) {
+        String[] parts = group.split(":");
+        Set<Text> cols = new HashSet<>();
+        for (String col : parts[1].split(",")) {
+          cols.add(t(col));
+        }
+        groups.put(parts[1], cols);
+      }
+    }
+    return groups;
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
index b1b7117..f14e8d5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -42,15 +43,12 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterControl;
@@ -58,6 +56,8 @@
 import org.apache.accumulo.test.util.Wait;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class ManagerAssignmentIT extends AccumuloClusterHarness {
 
   @Override
@@ -71,12 +71,11 @@
       String tableName = super.getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
       String tableId = c.tableOperations().tableIdMap().get(tableName);
+
       // wait for the table to be online
-      TabletLocationState newTablet;
-      do {
-        UtilWaitThread.sleep(250);
-        newTablet = getTabletLocationState(c, tableId);
-      } while (newTablet.current == null);
+      Wait.waitFor(() -> getTabletLocationState(c, tableId) != null, SECONDS.toMillis(60), 250);
+
+      TabletLocationState newTablet = getTabletLocationState(c, tableId);
       assertNull(newTablet.last);
       assertNull(newTablet.future);
 
@@ -122,7 +121,8 @@
 
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
 
-      Wait.waitFor(() -> client.instanceOperations().getTabletServers().size() == 1);
+      Wait.waitFor(() -> client.instanceOperations().getTabletServers().size() == 1,
+          SECONDS.toMillis(60), SECONDS.toMillis(2));
 
       client.tableOperations().create(tableName);
 
@@ -173,29 +173,28 @@
 
       Locations locs = client.tableOperations().locate(tableName,
           Collections.singletonList(TabletsSection.getRange()));
-      locs.groupByTablet().keySet().stream().map(tid -> locs.getTabletLocation(tid))
-          .forEach(location -> {
-            HostAndPort address = HostAndPort.fromString(location);
-            String addressWithSession = address.toString();
-            var zLockPath = ServiceLock.path(getCluster().getServerContext().getZooKeeperRoot()
-                + Constants.ZTSERVERS + "/" + address.toString());
-            long sessionId =
-                ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), zLockPath);
-            if (sessionId != 0) {
-              addressWithSession = address.toString() + "[" + Long.toHexString(sessionId) + "]";
-            }
+      locs.groupByTablet().keySet().stream().map(locs::getTabletLocation).forEach(location -> {
+        HostAndPort address = HostAndPort.fromString(location);
+        String addressWithSession = address.toString();
+        var zLockPath = ServiceLock.path(getCluster().getServerContext().getZooKeeperRoot()
+            + Constants.ZTSERVERS + "/" + address);
+        long sessionId =
+            ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), zLockPath);
+        if (sessionId != 0) {
+          addressWithSession = address + "[" + Long.toHexString(sessionId) + "]";
+        }
 
-            final String finalAddress = addressWithSession;
-            System.out.println("Attempting to shutdown TabletServer at: " + address.toString());
-            try {
-              ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
-                  c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
-                      getCluster().getServerContext().rpcCreds(), finalAddress, false));
-            } catch (AccumuloException | AccumuloSecurityException e) {
-              fail("Error shutting down TabletServer", e);
-            }
+        final String finalAddress = addressWithSession;
+        System.out.println("Attempting to shutdown TabletServer at: " + address);
+        try {
+          ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
+              c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
+                  getCluster().getServerContext().rpcCreds(), finalAddress, false));
+        } catch (AccumuloException | AccumuloSecurityException e) {
+          fail("Error shutting down TabletServer", e);
+        }
 
-          });
+      });
 
       Wait.waitFor(() -> client.instanceOperations().getTabletServers().size() == 0);
 
@@ -212,7 +211,8 @@
 
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
 
-      Wait.waitFor(() -> client.instanceOperations().getTabletServers().size() == 1, 60_000);
+      Wait.waitFor(() -> client.instanceOperations().getTabletServers().size() == 1,
+          SECONDS.toMillis(60), SECONDS.toMillis(2));
 
       client.instanceOperations().waitForBalance();
 
@@ -220,31 +220,30 @@
       // could potentially send a kill -9 to the process. Shut the tablet
       // servers down in a more graceful way.
 
-      Locations locs = client.tableOperations().locate(RootTable.NAME,
+      Locations locs = client.tableOperations().locate(AccumuloTable.ROOT.tableName(),
           Collections.singletonList(TabletsSection.getRange()));
-      locs.groupByTablet().keySet().stream().map(tid -> locs.getTabletLocation(tid))
-          .forEach(location -> {
-            HostAndPort address = HostAndPort.fromString(location);
-            String addressWithSession = address.toString();
-            var zLockPath = ServiceLock.path(getCluster().getServerContext().getZooKeeperRoot()
-                + Constants.ZTSERVERS + "/" + address.toString());
-            long sessionId =
-                ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), zLockPath);
-            if (sessionId != 0) {
-              addressWithSession = address.toString() + "[" + Long.toHexString(sessionId) + "]";
-            }
+      locs.groupByTablet().keySet().stream().map(locs::getTabletLocation).forEach(location -> {
+        HostAndPort address = HostAndPort.fromString(location);
+        String addressWithSession = address.toString();
+        var zLockPath = ServiceLock.path(getCluster().getServerContext().getZooKeeperRoot()
+            + Constants.ZTSERVERS + "/" + address);
+        long sessionId =
+            ServiceLock.getSessionId(getCluster().getServerContext().getZooCache(), zLockPath);
+        if (sessionId != 0) {
+          addressWithSession = address + "[" + Long.toHexString(sessionId) + "]";
+        }
 
-            final String finalAddress = addressWithSession;
-            System.out.println("Attempting to shutdown TabletServer at: " + address.toString());
-            try {
-              ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
-                  c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
-                      getCluster().getServerContext().rpcCreds(), finalAddress, false));
-            } catch (AccumuloException | AccumuloSecurityException e) {
-              fail("Error shutting down TabletServer", e);
-            }
+        final String finalAddress = addressWithSession;
+        System.out.println("Attempting to shutdown TabletServer at: " + address);
+        try {
+          ThriftClientTypes.MANAGER.executeVoid((ClientContext) client,
+              c -> c.shutdownTabletServer(TraceUtil.traceInfo(),
+                  getCluster().getServerContext().rpcCreds(), finalAddress, false));
+        } catch (AccumuloException | AccumuloSecurityException e) {
+          fail("Error shutting down TabletServer", e);
+        }
 
-          });
+      });
 
       Wait.waitFor(() -> client.instanceOperations().getTabletServers().size() == 0);
 
@@ -253,7 +252,8 @@
 
   private TabletLocationState getTabletLocationState(AccumuloClient c, String tableId) {
     try (MetaDataTableScanner s = new MetaDataTableScanner((ClientContext) c,
-        new Range(TabletsSection.encodeRow(TableId.of(tableId), null)), MetadataTable.NAME)) {
+        new Range(TabletsSection.encodeRow(TableId.of(tableId), null)),
+        AccumuloTable.METADATA.tableName())) {
       return s.next();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
index fe4b792..12ff411 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.Base64;
@@ -151,7 +152,7 @@
           for (int j = 0; j < 10; j++) {
             int row = startRow + j;
             Mutation m = new Mutation(String.format("%05x", row));
-            random.nextBytes(val);
+            RANDOM.get().nextBytes(val);
             m.put("f", "q", "v");
 
             manyWALsWriter.addMutation(m);
@@ -161,7 +162,7 @@
           // write a lot of data to second table to forces the logs to roll
           for (int j = 0; j < 1000; j++) {
             Mutation m = new Mutation(String.format("%03d", j));
-            random.nextBytes(val);
+            RANDOM.get().nextBytes(val);
 
             m.put("f", "q", Base64.getEncoder().encodeToString(val));
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
index b576411..8e1a2f5 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MaxOpenIT.java
@@ -48,7 +48,6 @@
  * A functional test that exercises hitting the max open file limit on a tablet server. This test
  * assumes there are one or two tablet servers.
  */
-@SuppressWarnings("removal")
 public class MaxOpenIT extends AccumuloClusterHarness {
 
   @Override
@@ -60,12 +59,10 @@
   public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     Map<String,String> conf = cfg.getSiteConfig();
     conf.put(Property.TSERV_SCAN_MAX_OPENFILES.getKey(), "4");
-    conf.put(Property.TSERV_MAJC_MAXCONCURRENT.getKey(), "1");
-    conf.put(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), "2");
     cfg.setSiteConfig(conf);
   }
 
-  private String scanMaxOpenFiles, majcConcurrent, majcThreadMaxOpen;
+  private String scanMaxOpenFiles;
 
   @BeforeEach
   public void alterConfig() throws Exception {
@@ -73,8 +70,6 @@
       InstanceOperations iops = client.instanceOperations();
       Map<String,String> sysConfig = iops.getSystemConfiguration();
       scanMaxOpenFiles = sysConfig.get(Property.TSERV_SCAN_MAX_OPENFILES.getKey());
-      majcConcurrent = sysConfig.get(Property.TSERV_MAJC_MAXCONCURRENT.getKey());
-      majcThreadMaxOpen = sysConfig.get(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey());
     }
   }
 
@@ -85,12 +80,6 @@
       if (scanMaxOpenFiles != null) {
         iops.setProperty(Property.TSERV_SCAN_MAX_OPENFILES.getKey(), scanMaxOpenFiles);
       }
-      if (majcConcurrent != null) {
-        iops.setProperty(Property.TSERV_MAJC_MAXCONCURRENT.getKey(), majcConcurrent);
-      }
-      if (majcThreadMaxOpen != null) {
-        iops.setProperty(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey(), majcThreadMaxOpen);
-      }
     }
   }
 
@@ -107,7 +96,7 @@
           .withSplits(TestIngest.getSplitPoints(0, NUM_TO_INGEST, NUM_TABLETS));
       c.tableOperations().create(tableName, ntc);
 
-      // the following loop should create three tablets in each map file
+      // the following loop should create three tablets in each data file
       for (int i = 0; i < 3; i++) {
         IngestParams params = new IngestParams(getClientProps(), tableName, NUM_TO_INGEST);
         params.timestamp = i;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryConsumingIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryConsumingIterator.java
new file mode 100644
index 0000000..2b23fb1
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryConsumingIterator.java
@@ -0,0 +1,89 @@
+/*
+ * 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.test.functional;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class MemoryConsumingIterator extends WrappingIterator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MemoryConsumingIterator.class);
+
+  private static final List<byte[]> BUFFERS = new ArrayList<>();
+
+  public static void freeBuffers() {
+    BUFFERS.clear();
+  }
+
+  @SuppressFBWarnings(value = "DM_GC", justification = "gc is okay for test")
+  private int getAmountToConsume() {
+    System.gc();
+    Runtime runtime = Runtime.getRuntime();
+    long maxConfiguredMemory = runtime.maxMemory();
+    long usedMemory = runtime.totalMemory() - runtime.freeMemory();
+    long freeMemory = maxConfiguredMemory - usedMemory;
+    long minFreeMemory = (long) (maxConfiguredMemory * MemoryStarvedScanIT.FREE_MEMORY_THRESHOLD);
+
+    // consume free memory, and exceed the minimum threshold by just a little bit
+    // don't exceed typical JDK byte array limit
+    long amountToConsume =
+        Math.min(Math.max(0, freeMemory + 1 - minFreeMemory), Integer.MAX_VALUE - 8);
+    LOG.info("max: {}, used: {}, free: {}, minFree: {}, amountToConsume: {}", maxConfiguredMemory,
+        usedMemory, freeMemory, minFreeMemory, amountToConsume);
+    return (int) amountToConsume;
+  }
+
+  @Override
+  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
+      throws IOException {
+    LOG.info("seek called");
+    while (!this.isRunningLowOnMemory()) {
+      int amountToConsume = getAmountToConsume();
+      if (amountToConsume > 0) {
+        LOG.info("allocating memory: " + amountToConsume);
+        BUFFERS.add(new byte[amountToConsume]);
+        LOG.info("memory allocated");
+      } else {
+        LOG.info("consumed enough; no more memory allocated");
+      }
+      LOG.info("Waiting for LowMemoryDetector to recognize low on memory condition.");
+      try {
+        Thread.sleep(SECONDS.toMillis(5));
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new IOException("interrupted during sleep", ex);
+      }
+    }
+    LOG.info("Running low on memory == true");
+    super.seek(range, columnFamilies, inclusive);
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryFreeingIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryFreeingIterator.java
new file mode 100644
index 0000000..95e7e2e
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryFreeingIterator.java
@@ -0,0 +1,44 @@
+/*
+ * 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.test.functional;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class MemoryFreeingIterator extends WrappingIterator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MemoryFreeingIterator.class);
+
+  @SuppressFBWarnings(value = "DM_GC", justification = "gc is okay for test")
+  public MemoryFreeingIterator() throws InterruptedException {
+    LOG.info("Try to free consumed memory - will block until isRunningLowOnMemory returns false.");
+    MemoryConsumingIterator.freeBuffers();
+    while (this.isRunningLowOnMemory()) {
+      System.gc();
+      // wait for LowMemoryDetector to recognize the memory is free.
+      Thread.sleep(SECONDS.toMillis(1));
+    }
+    LOG.info("isRunningLowOnMemory returned false - memory available");
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java
new file mode 100644
index 0000000..771d74d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java
@@ -0,0 +1,162 @@
+/*
+ * 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.test.functional;
+
+import static org.apache.accumulo.test.util.Wait.waitFor;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.DoubleAdder;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.metrics.TestStatsDRegistryFactory;
+import org.apache.accumulo.test.metrics.TestStatsDSink;
+import org.apache.accumulo.test.metrics.TestStatsDSink.Metric;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class MemoryStarvedMajCIT extends SharedMiniClusterBase {
+
+  public static class MemoryStarvedITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      cfg.setNumTservers(1);
+      cfg.setMemory(ServerType.TABLET_SERVER, 256, MemoryUnit.MEGABYTE);
+      // Configure the LowMemoryDetector in the TabletServer
+      cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_INTERVAL, "5s");
+      cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_THRESHOLD,
+          Double.toString(MemoryStarvedScanIT.FREE_MEMORY_THRESHOLD));
+      cfg.setProperty(Property.GENERAL_LOW_MEM_MAJC_PROTECTION, "true");
+      // Tell the server processes to use a StatsDMeterRegistry that will be configured
+      // to push all metrics to the sink we started.
+      cfg.setProperty(Property.GENERAL_MICROMETER_ENABLED, "true");
+      cfg.setProperty(Property.GENERAL_MICROMETER_FACTORY,
+          TestStatsDRegistryFactory.class.getName());
+      Map<String,String> sysProps = Map.of(TestStatsDRegistryFactory.SERVER_HOST, "127.0.0.1",
+          TestStatsDRegistryFactory.SERVER_PORT, Integer.toString(sink.getPort()));
+      cfg.setSystemProperties(sysProps);
+    }
+  }
+
+  private static final DoubleAdder MAJC_PAUSED = new DoubleAdder();
+  private static TestStatsDSink sink;
+  private static Thread metricConsumer;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    sink = new TestStatsDSink();
+    metricConsumer = new Thread(() -> {
+      while (!Thread.currentThread().isInterrupted()) {
+        List<String> statsDMetrics = sink.getLines();
+        for (String line : statsDMetrics) {
+          if (Thread.currentThread().isInterrupted()) {
+            break;
+          }
+          if (line.startsWith("accumulo")) {
+            Metric metric = TestStatsDSink.parseStatsDMetric(line);
+            if (MetricsProducer.METRICS_MAJC_PAUSED.equals(metric.getName())) {
+              double val = Double.parseDouble(metric.getValue());
+              MAJC_PAUSED.add(val);
+            }
+          }
+        }
+      }
+    });
+    metricConsumer.start();
+
+    SharedMiniClusterBase.startMiniClusterWithConfig(new MemoryStarvedITConfiguration());
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+    sink.close();
+    metricConsumer.interrupt();
+    metricConsumer.join();
+  }
+
+  @BeforeEach
+  public void beforeEach() {
+    // Reset the client side counters
+    MAJC_PAUSED.reset();
+  }
+
+  @Test
+  public void testMajCPauses() throws Exception {
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      // Add a small amount of data so that the MemoryConsumingIterator
+      // returns true when trying to consume all of the memory.
+      ReadWriteIT.ingest(client, 1, 1, 1, 0, table);
+
+      AtomicReference<Throwable> error = new AtomicReference<>();
+      Thread compactionThread = new Thread(() -> {
+        try {
+          to.compact(table, new CompactionConfig().setWait(false));
+        } catch (Exception e) {
+          error.set(e);
+        }
+      });
+
+      try (Scanner scanner = client.createScanner(table)) {
+
+        MemoryStarvedScanIT.consumeServerMemory(scanner);
+
+        int paused = MAJC_PAUSED.intValue();
+        assertEquals(0, paused);
+
+        ReadWriteIT.ingest(client, 100, 100, 100, 0, table);
+        compactionThread.start();
+
+        waitFor(() -> MAJC_PAUSED.intValue() > 0);
+
+        MemoryStarvedScanIT.freeServerMemory(client);
+        compactionThread.interrupt();
+        compactionThread.join();
+        assertNull(error.get());
+        assertTrue(client.instanceOperations().getActiveCompactions().stream()
+            .anyMatch(ac -> ac.getPausedCount() > 0));
+      }
+    }
+
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java
new file mode 100644
index 0000000..edb4cd0
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMinCIT.java
@@ -0,0 +1,163 @@
+/*
+ * 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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.DoubleAdder;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.metrics.TestStatsDRegistryFactory;
+import org.apache.accumulo.test.metrics.TestStatsDSink;
+import org.apache.accumulo.test.metrics.TestStatsDSink.Metric;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class MemoryStarvedMinCIT extends SharedMiniClusterBase {
+
+  public static class MemoryStarvedITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      cfg.setNumTservers(1);
+      cfg.setMemory(ServerType.TABLET_SERVER, 256, MemoryUnit.MEGABYTE);
+      // Configure the LowMemoryDetector in the TabletServer
+      cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_INTERVAL, "5s");
+      cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_THRESHOLD,
+          Double.toString(MemoryStarvedScanIT.FREE_MEMORY_THRESHOLD));
+      cfg.setProperty(Property.GENERAL_LOW_MEM_MINC_PROTECTION, "true");
+      // Tell the server processes to use a StatsDMeterRegistry that will be configured
+      // to push all metrics to the sink we started.
+      cfg.setProperty(Property.GENERAL_MICROMETER_ENABLED, "true");
+      cfg.setProperty(Property.GENERAL_MICROMETER_FACTORY,
+          TestStatsDRegistryFactory.class.getName());
+      Map<String,String> sysProps = Map.of(TestStatsDRegistryFactory.SERVER_HOST, "127.0.0.1",
+          TestStatsDRegistryFactory.SERVER_PORT, Integer.toString(sink.getPort()));
+      cfg.setSystemProperties(sysProps);
+    }
+  }
+
+  private static final DoubleAdder MINC_PAUSED = new DoubleAdder();
+  private static TestStatsDSink sink;
+  private static Thread metricConsumer;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    sink = new TestStatsDSink();
+    metricConsumer = new Thread(() -> {
+      while (!Thread.currentThread().isInterrupted()) {
+        List<String> statsDMetrics = sink.getLines();
+        for (String line : statsDMetrics) {
+          if (Thread.currentThread().isInterrupted()) {
+            break;
+          }
+          if (line.startsWith("accumulo")) {
+            Metric metric = TestStatsDSink.parseStatsDMetric(line);
+            if (MetricsProducer.METRICS_MINC_PAUSED.equals(metric.getName())) {
+              double val = Double.parseDouble(metric.getValue());
+              MINC_PAUSED.add(val);
+            }
+          }
+        }
+      }
+    });
+    metricConsumer.start();
+
+    SharedMiniClusterBase.startMiniClusterWithConfig(new MemoryStarvedITConfiguration());
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+    sink.close();
+    metricConsumer.interrupt();
+    metricConsumer.join();
+  }
+
+  @BeforeEach
+  public void beforeEach() {
+    // Reset the client side counters
+    MINC_PAUSED.reset();
+  }
+
+  @Test
+  public void testMinCPauses() throws Exception {
+
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      // Add a small amount of data so that the MemoryConsumingIterator
+      // returns true when trying to consume all of the memory.
+      ReadWriteIT.ingest(client, 1, 1, 1, 0, table);
+
+      AtomicReference<Throwable> error = new AtomicReference<>();
+      Thread ingestThread = new Thread(() -> {
+        try {
+          ReadWriteIT.ingest(client, 100, 100, 100, 0, table);
+          to.flush(table);
+        } catch (Exception e) {
+          error.set(e);
+        }
+      });
+
+      try (Scanner scanner = client.createScanner(table)) {
+
+        MemoryStarvedScanIT.consumeServerMemory(scanner);
+
+        int paused = MINC_PAUSED.intValue();
+        assertEquals(0, paused);
+
+        ingestThread.start();
+
+        while (paused <= 0) {
+          Thread.sleep(1000);
+          paused = MINC_PAUSED.intValue();
+        }
+
+        MemoryStarvedScanIT.freeServerMemory(client);
+        ingestThread.interrupt();
+        ingestThread.join();
+        assertNull(error.get());
+        assertTrue(client.instanceOperations().getActiveCompactions().stream()
+            .anyMatch(ac -> ac.getPausedCount() > 0));
+      }
+    }
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java
new file mode 100644
index 0000000..d2cb595
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedScanIT.java
@@ -0,0 +1,532 @@
+/*
+ * 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.test.functional;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.metrics.MetricsProducer.METRICS_LOW_MEMORY;
+import static org.apache.accumulo.test.util.Wait.waitFor;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.DoubleAdder;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.conf.Property;
+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.iterators.WrappingIterator;
+import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.core.spi.metrics.LoggingMeterRegistryFactory;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.MemoryUnit;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.metrics.TestStatsDRegistryFactory;
+import org.apache.accumulo.test.metrics.TestStatsDSink;
+import org.apache.accumulo.test.metrics.TestStatsDSink.Metric;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MemoryStarvedScanIT extends SharedMiniClusterBase {
+
+  public static class MemoryStarvedITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      cfg.setNumTservers(1);
+      cfg.setMemory(ServerType.TABLET_SERVER, 256, MemoryUnit.MEGABYTE);
+      // Configure the LowMemoryDetector in the TabletServer
+      cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_INTERVAL, "5s");
+      cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_THRESHOLD,
+          Double.toString(FREE_MEMORY_THRESHOLD));
+      cfg.setProperty(Property.GENERAL_LOW_MEM_SCAN_PROTECTION, "true");
+      // Tell the server processes to use a StatsDMeterRegistry that will be configured
+      // to push all metrics to the sink we started.
+      cfg.setProperty(Property.GENERAL_MICROMETER_ENABLED, "true");
+      cfg.setProperty("general.custom.metrics.opts.logging.step", "5s");
+      String clazzList = LoggingMeterRegistryFactory.class.getName() + ","
+          + TestStatsDRegistryFactory.class.getName();
+      cfg.setProperty(Property.GENERAL_MICROMETER_FACTORY, clazzList);
+      Map<String,String> sysProps = Map.of(TestStatsDRegistryFactory.SERVER_HOST, "127.0.0.1",
+          TestStatsDRegistryFactory.SERVER_PORT, Integer.toString(sink.getPort()));
+      cfg.setSystemProperties(sysProps);
+    }
+  }
+
+  public static final double FREE_MEMORY_THRESHOLD = 0.40D;
+
+  private static final Logger LOG = LoggerFactory.getLogger(MemoryStarvedScanIT.class);
+  private static final DoubleAdder SCAN_START_DELAYED = new DoubleAdder();
+  private static final DoubleAdder SCAN_RETURNED_EARLY = new DoubleAdder();
+  private static final AtomicInteger LOW_MEM_DETECTED = new AtomicInteger(0);
+  private static TestStatsDSink sink;
+  private static Thread metricConsumer;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    sink = new TestStatsDSink();
+    metricConsumer = new Thread(() -> {
+      while (!Thread.currentThread().isInterrupted()) {
+        List<String> statsDMetrics = sink.getLines();
+        for (String line : statsDMetrics) {
+          if (Thread.currentThread().isInterrupted()) {
+            break;
+          }
+          if (line.startsWith("accumulo")) {
+            Metric metric = TestStatsDSink.parseStatsDMetric(line);
+            if (MetricsProducer.METRICS_SCAN_PAUSED_FOR_MEM.equals(metric.getName())) {
+              double val = Double.parseDouble(metric.getValue());
+              SCAN_START_DELAYED.add(val);
+            } else if (MetricsProducer.METRICS_SCAN_RETURN_FOR_MEM.equals(metric.getName())) {
+              double val = Double.parseDouble(metric.getValue());
+              SCAN_RETURNED_EARLY.add(val);
+            } else if (metric.getName().equals(METRICS_LOW_MEMORY)) {
+              String process = metric.getTags().get("process.name");
+              if (process != null && process.contains("tserver")) {
+                int val = Integer.parseInt(metric.getValue());
+                LOW_MEM_DETECTED.set(val);
+              }
+            }
+          }
+        }
+      }
+    });
+    metricConsumer.start();
+
+    SharedMiniClusterBase.startMiniClusterWithConfig(new MemoryStarvedITConfiguration());
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+    sink.close();
+    metricConsumer.interrupt();
+    metricConsumer.join();
+  }
+
+  @BeforeEach
+  public void beforeEach() throws Exception {
+    // Free the server side memory
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      freeServerMemory(client);
+    }
+    // allow metric collection to cycle and metric value to reset to zero
+    waitFor(() -> 0 == LOW_MEM_DETECTED.get());
+    // Reset the client side counters
+    SCAN_START_DELAYED.reset();
+    SCAN_START_DELAYED.reset();
+  }
+
+  static void consumeServerMemory(Scanner scanner) {
+    // This iterator will attempt to consume all free memory in the TabletServer
+    scanner.addScanIterator(new IteratorSetting(11, MemoryConsumingIterator.class, Map.of()));
+    scanner.setBatchSize(1);
+    // Set the ReadaheadThreshold to a large number so that another background thread
+    // that performs read-ahead of KV pairs is not started.
+    scanner.setReadaheadThreshold(Long.MAX_VALUE);
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    // This should block until the GarbageCollectionLogger runs and notices that the
+    // VM is low on memory.
+    assertTrue(iter.hasNext());
+  }
+
+  private void consumeServerMemory(BatchScanner scanner) {
+    // This iterator will attempt to consume all free memory in the TabletServer
+    scanner.addScanIterator(new IteratorSetting(11, MemoryConsumingIterator.class, Map.of()));
+    scanner.setRanges(Collections.singletonList(new Range()));
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    // This should block until the GarbageCollectionLogger runs and notices that the
+    // VM is low on memory.
+    assertTrue(iter.hasNext());
+  }
+
+  static void freeServerMemory(AccumuloClient client) throws Exception {
+    // Instantiating this class on the TabletServer will free the memory as it
+    // frees the buffers created by the MemoryConsumingIterator in its constructor.
+    client.instanceOperations().testClassLoad(MemoryFreeingIterator.class.getName(),
+        WrappingIterator.class.getName());
+  }
+
+  @Test
+  public void testScanReturnsEarlyDueToLowMemory() throws Exception {
+
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      ReadWriteIT.ingest(client, 10, 10, 10, 0, table);
+
+      try (Scanner scanner = client.createScanner(table)) {
+        final double returned = SCAN_RETURNED_EARLY.doubleValue();
+        final double paused = SCAN_START_DELAYED.doubleValue();
+
+        consumeServerMemory(scanner);
+
+        // Wait for The metric that indicates a scan was returned early due to low memory
+        waitFor(() -> SCAN_RETURNED_EARLY.doubleValue() > returned
+            && SCAN_START_DELAYED.doubleValue() >= paused);
+
+        freeServerMemory(client);
+      } finally {
+        to.delete(table);
+      }
+    }
+  }
+
+  @Test
+  public void testScanPauses() throws Exception {
+
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      ReadWriteIT.ingest(client, 10, 3, 10, 0, table);
+
+      try (Scanner dataConsumingScanner = client.createScanner(table);
+          Scanner memoryConsumingScanner = client.createScanner(table)) {
+
+        dataConsumingScanner.addScanIterator(
+            new IteratorSetting(11, SlowIterator.class, Map.of("sleepTime", "2000")));
+        dataConsumingScanner.setBatchSize(1);
+        dataConsumingScanner.setReadaheadThreshold(Long.MAX_VALUE);
+        Iterator<Entry<Key,Value>> iter = dataConsumingScanner.iterator();
+        AtomicInteger fetched = new AtomicInteger(0);
+        Thread t = new Thread(() -> {
+          while (iter.hasNext()) {
+            iter.next();
+            fetched.incrementAndGet();
+          }
+        });
+
+        memoryConsumingScanner
+            .addScanIterator(new IteratorSetting(11, MemoryConsumingIterator.class, Map.of()));
+        memoryConsumingScanner.setBatchSize(1);
+        memoryConsumingScanner.setReadaheadThreshold(Long.MAX_VALUE);
+
+        t.start();
+        LOG.info("Waiting for memory to be consumed");
+
+        // Wait until the dataConsumingScanner has started fetching data
+        int currentCount = fetched.get();
+        while (currentCount == 0) {
+          Thread.sleep(500);
+          currentCount = fetched.get();
+        }
+
+        // This should block until the GarbageCollectionLogger runs and notices that the
+        // VM is low on memory.
+        Iterator<Entry<Key,Value>> consumingIter = memoryConsumingScanner.iterator();
+        assertTrue(consumingIter.hasNext());
+
+        // Confirm that some data was fetched by the memoryConsumingScanner
+        currentCount = fetched.get();
+        assertTrue(currentCount > 0 && currentCount < 100);
+        LOG.info("Memory consumed after reading {} rows", currentCount);
+
+        // Grab the current metric counts, wait
+        final double returned = SCAN_RETURNED_EARLY.doubleValue();
+        final double paused = SCAN_START_DELAYED.doubleValue();
+        Thread.sleep(1500);
+        // One of two conditions could exist here:
+        // The number of fetched rows equals the current count before the wait above
+        // and the SCAN_START_DELAYED has been incremented OR the number of fetched
+        // rows is one more than the current count and the SCAN_RETURNED_EARLY has
+        // been incremented.
+        final int currentCountCopy = currentCount;
+        waitFor(
+            () -> (currentCountCopy == fetched.get() && SCAN_START_DELAYED.doubleValue() > paused)
+                || (currentCountCopy + 1 == fetched.get()
+                    && SCAN_RETURNED_EARLY.doubleValue() > returned));
+        currentCount = fetched.get();
+
+        // Perform the check again
+        waitFor(() -> 1 == LOW_MEM_DETECTED.get());
+        assertEquals(currentCount, fetched.get());
+
+        // Free the memory which will allow the pausing scanner to continue
+        LOG.info("Freeing memory");
+        freeServerMemory(client);
+        LOG.info("Memory freed");
+
+        t.join();
+        assertEquals(30, fetched.get());
+      } finally {
+        to.delete(table);
+      }
+    }
+  }
+
+  @Test
+  public void testBatchScanReturnsEarlyDueToLowMemory() throws Exception {
+
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      // check memory okay before starting
+      assertEquals(0, LOW_MEM_DETECTED.get());
+
+      ReadWriteIT.ingest(client, 10, 10, 10, 0, table);
+
+      try (BatchScanner scanner = client.createBatchScanner(table,
+          client.securityOperations().getUserAuthorizations(client.whoami()), 1)) {
+
+        final double returned = SCAN_RETURNED_EARLY.doubleValue();
+        final double paused = SCAN_START_DELAYED.doubleValue();
+
+        consumeServerMemory(scanner);
+
+        // Wait for metric that indicates a scan was returned early due to low memory
+        waitFor(() -> SCAN_RETURNED_EARLY.doubleValue() > returned
+            && SCAN_START_DELAYED.doubleValue() >= paused);
+        waitFor(() -> 1 == LOW_MEM_DETECTED.get());
+
+        freeServerMemory(client);
+      } finally {
+        to.delete(table);
+      }
+    }
+  }
+
+  @Test
+  public void testBatchScanPauses() throws Exception {
+
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      // check memory okay before starting
+      assertEquals(0, LOW_MEM_DETECTED.get());
+
+      // generate enough data so more than one batch is returned.
+      ReadWriteIT.ingest(client, 1000, 3, 10, 0, table);
+
+      try (BatchScanner dataConsumingScanner = client.createBatchScanner(table);
+          Scanner memoryConsumingScanner = client.createScanner(table)) {
+
+        // add enough delay that batch does not return all rows and we get more than one batch
+        dataConsumingScanner.addScanIterator(
+            new IteratorSetting(11, SlowIterator.class, Map.of("sleepTime", "50")));
+        dataConsumingScanner.setRanges(Collections.singletonList(new Range()));
+        Iterator<Entry<Key,Value>> iter = dataConsumingScanner.iterator();
+        AtomicInteger fetched = new AtomicInteger(0);
+        Thread t = new Thread(() -> {
+          while (iter.hasNext()) {
+            iter.next();
+            fetched.incrementAndGet();
+          }
+        });
+
+        memoryConsumingScanner
+            .addScanIterator(new IteratorSetting(11, MemoryConsumingIterator.class, Map.of()));
+        memoryConsumingScanner.setBatchSize(1);
+        memoryConsumingScanner.setReadaheadThreshold(Long.MAX_VALUE);
+
+        t.start();
+
+        // Wait for a batch to be returned
+        waitFor(() -> fetched.get() > 0, MINUTES.toMillis(5), 200);
+
+        // This should block until the GarbageCollectionLogger runs and notices that the
+        // VM is low on memory.
+        Iterator<Entry<Key,Value>> consumingIter = memoryConsumingScanner.iterator();
+        assertTrue(consumingIter.hasNext());
+
+        // Grab the current paused count, the number of rows fetched by the memoryConsumingScanner
+        // has not increased
+        // and that the scan delay counter has increased.
+        final double returned = SCAN_RETURNED_EARLY.doubleValue();
+        final double paused = SCAN_START_DELAYED.doubleValue();
+
+        // Confirm that some data was fetched by the dataConsumingScanner
+        int currentCount = fetched.get();
+        LOG.info("rows read in first batch: {}", currentCount);
+        // check some, but not all rows have been read
+        assertTrue(currentCount > 0 && currentCount < 3000);
+
+        final int startCount = currentCount;
+        waitFor(() -> verifyBatchedStalled(fetched.get(), startCount, paused, returned),
+            MINUTES.toMillis(2), SECONDS.toMillis(2));
+        waitFor(() -> 1 == LOW_MEM_DETECTED.get());
+
+        // Perform the check again - checking that rows fetched not advancing
+        final double paused2 = SCAN_START_DELAYED.doubleValue();
+        final double returned2 = SCAN_RETURNED_EARLY.doubleValue();
+        Thread.sleep(1500);
+        assertEquals(startCount, fetched.get());
+        assertTrue(SCAN_START_DELAYED.doubleValue() >= paused2);
+        assertEquals(returned2, SCAN_RETURNED_EARLY.doubleValue());
+        waitFor(() -> 1 == LOW_MEM_DETECTED.get());
+
+        // Free the memory which will allow the pausing scanner to continue
+        freeServerMemory(client);
+
+        waitFor(() -> 0 == LOW_MEM_DETECTED.get());
+
+        t.join();
+        // check that remain rows have been read
+        assertEquals(3000, fetched.get());
+
+      } finally {
+        to.delete(table);
+      }
+    }
+  }
+
+  private boolean verifyBatchedStalled(final int currCount, final int startCount,
+      final double paused, final double returned) {
+    if (startCount == currCount && SCAN_START_DELAYED.doubleValue() > paused) {
+      LOG.debug("found expected pause because of low memory");
+      return true;
+    }
+    if (startCount == currCount && SCAN_RETURNED_EARLY.doubleValue() > returned) {
+      LOG.debug("found expected early return because of low memory");
+      return true;
+    }
+    LOG.info(
+        "waiting for low memory pause. prev count: {}, curr count: {}, paused: {}, returned: {}",
+        startCount, currCount, SCAN_START_DELAYED.doubleValue(), SCAN_RETURNED_EARLY.doubleValue());
+    return false;
+  }
+
+  /**
+   * Check that the low memory condition is set and remains set until free memory is available.
+   */
+  @Test
+  public void testLowMemoryFlapping() throws Exception {
+
+    String table = getUniqueNames(1)[0];
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      TableOperations to = client.tableOperations();
+      to.create(table);
+
+      // check memory okay before starting
+      assertEquals(0, LOW_MEM_DETECTED.get());
+
+      ReadWriteIT.ingest(client, 10, 3, 10, 0, table);
+
+      try (BatchScanner dataConsumingScanner = client.createBatchScanner(table);
+          Scanner memoryConsumingScanner = client.createScanner(table)) {
+
+        dataConsumingScanner.addScanIterator(
+            new IteratorSetting(11, SlowIterator.class, Map.of("sleepTime", "500")));
+        dataConsumingScanner.setRanges(Collections.singletonList(new Range()));
+        Iterator<Entry<Key,Value>> iter = dataConsumingScanner.iterator();
+        AtomicInteger fetched = new AtomicInteger(0);
+        Thread t = new Thread(() -> {
+          int i = 0;
+          while (iter.hasNext()) {
+            iter.next();
+            fetched.set(++i);
+          }
+        });
+
+        memoryConsumingScanner
+            .addScanIterator(new IteratorSetting(11, MemoryConsumingIterator.class, Map.of()));
+        memoryConsumingScanner.setBatchSize(1);
+        memoryConsumingScanner.setReadaheadThreshold(Long.MAX_VALUE);
+
+        t.start();
+
+        // Wait until the dataConsumingScanner has started fetching data
+        int currentCount = fetched.get();
+        while (currentCount == 0) {
+          Thread.sleep(500);
+          currentCount = fetched.get();
+        }
+
+        // This should block until the GarbageCollectionLogger runs and notices that the
+        // VM is low on memory.
+        Iterator<Entry<Key,Value>> consumingIter = memoryConsumingScanner.iterator();
+        assertTrue(consumingIter.hasNext());
+
+        // Confirm that some data was fetched by the dataConsumingScanner
+        currentCount = fetched.get();
+        assertTrue(currentCount > 0 && currentCount < 100);
+
+        // Grab the current paused count, wait two seconds and then confirm that
+        // the number of rows fetched by the memoryConsumingScanner has not increased
+        // and that the scan delay counter has increased.
+        double returned = SCAN_RETURNED_EARLY.doubleValue();
+        double paused = SCAN_START_DELAYED.doubleValue();
+        Thread.sleep(1500);
+        assertEquals(currentCount, fetched.get());
+        assertTrue(SCAN_START_DELAYED.doubleValue() >= paused);
+        assertTrue(SCAN_RETURNED_EARLY.doubleValue() >= returned);
+        waitFor(() -> LOW_MEM_DETECTED.get() == 1);
+
+        // Perform the check again
+        paused = SCAN_START_DELAYED.doubleValue();
+        returned = SCAN_RETURNED_EARLY.doubleValue();
+        Thread.sleep(1500);
+        assertEquals(currentCount, fetched.get());
+        assertTrue(SCAN_START_DELAYED.doubleValue() >= paused);
+        assertEquals(returned, SCAN_RETURNED_EARLY.doubleValue());
+
+        // check across multiple low memory checks and metric updates that low memory detected
+        // remains set
+        int checkCount = 0;
+        while (checkCount++ < 5) {
+          Thread.sleep(5_000);
+          LOG.debug("Check low memory still set. Low Memory Flag: {}, Check count: {}",
+              LOW_MEM_DETECTED.get(), checkCount);
+          assertEquals(1, LOW_MEM_DETECTED.get());
+        }
+        // Free the memory which will allow the pausing scanner to continue
+        freeServerMemory(client);
+
+        t.join();
+        assertEquals(30, fetched.get());
+        // allow metric collection to cycle.
+        waitFor(() -> LOW_MEM_DETECTED.get() == 0);
+
+      } finally {
+        to.delete(table);
+      }
+    }
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
index 43590e80..f63a2a9 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
@@ -18,16 +18,21 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.test.util.FileMetadataUtil.printAndVerifyFileMetadata;
+import static org.apache.accumulo.test.util.FileMetadataUtil.verifyMergedMarkerCleared;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 import java.time.Duration;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
@@ -36,31 +41,45 @@
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.conf.Property;
 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.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+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.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionMetadata;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
 import org.apache.accumulo.core.spi.compaction.CompactionKind;
+import org.apache.accumulo.core.util.FastFormat;
 import org.apache.accumulo.core.util.Merge;
 import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.TestIngest.IngestParams;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.accumulo.test.VerifyIngest.VerifyParams;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class MergeIT extends AccumuloClusterHarness {
 
+  private static final Logger log = LoggerFactory.getLogger(MergeIT.class);
+
   SortedSet<Text> splits(String[] points) {
     SortedSet<Text> result = new TreeSet<>();
     for (String point : points) {
@@ -79,7 +98,7 @@
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
       var ntc = new NewTableConfiguration().withSplits(splits("a b c d e f g h i j k".split(" ")));
-      c.tableOperations().create(tableName, ntc);
+      createTableAndDisableCompactions(c, tableName, ntc);
       try (BatchWriter bw = c.createBatchWriter(tableName)) {
         for (String row : "a b c d e f g h i j k".split(" ")) {
           Mutation m = new Mutation(row);
@@ -90,6 +109,9 @@
       c.tableOperations().flush(tableName, null, null, true);
       c.tableOperations().merge(tableName, new Text("c1"), new Text("f1"));
       assertEquals(8, c.tableOperations().listSplits(tableName).size());
+      // Verify that the MERGED marker was cleared
+      verifyMergedMarkerCleared(getServerContext(),
+          TableId.of(c.tableOperations().tableIdMap().get(tableName)));
     }
   }
 
@@ -99,7 +121,7 @@
       String tableName = getUniqueNames(1)[0];
       NewTableConfiguration ntc = new NewTableConfiguration()
           .withSplits(splits("a b c d e f g h i j k l m n o p q r s t u v w x y z".split(" ")));
-      c.tableOperations().create(tableName, ntc);
+      createTableAndDisableCompactions(c, tableName, ntc);
       try (BatchWriter bw = c.createBatchWriter(tableName)) {
         for (String row : "c e f y".split(" ")) {
           Mutation m = new Mutation(row);
@@ -117,6 +139,322 @@
     }
   }
 
+  @Test
+  public void noChopMergeTest() throws Exception {
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      createTableAndDisableCompactions(c, tableName, new NewTableConfiguration());
+      final TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      // First write 1000 rows to a file in the default tablet
+      ingest(c, 1000, 1, tableName);
+      c.tableOperations().flush(tableName, null, null, true);
+
+      log.debug("Metadata after Ingest");
+      printAndVerifyFileMetadata(getServerContext(), tableId, 1);
+
+      // Add splits so we end up with 4 tablets
+      final SortedSet<Text> splits = new TreeSet<>();
+      for (int i = 250; i <= 750; i += 250) {
+        splits.add(new Text("row_" + String.format("%010d", i)));
+      }
+      c.tableOperations().addSplits(tableName, splits);
+
+      log.debug("Metadata after Split");
+      verify(c, 1000, 1, tableName);
+      printAndVerifyFileMetadata(getServerContext(), tableId, 4);
+
+      // Go through and delete two blocks of rows, 101 - 200
+      // and also 301 - 400 so we can test that the data doesn't come
+      // back on merge
+      try (BatchWriter bw = c.createBatchWriter(tableName)) {
+        byte[] COL_PREFIX = "col_".getBytes(UTF_8);
+        Text colq = new Text(FastFormat.toZeroPaddedString(0, 7, 10, COL_PREFIX));
+
+        for (int i = 101; i <= 200; i++) {
+          Mutation m = new Mutation(new Text("row_" + String.format("%010d", i)));
+          m.putDelete(new Text("colf"), colq);
+          bw.addMutation(m);
+        }
+        for (int i = 301; i <= 400; i++) {
+          Mutation m = new Mutation(new Text("row_" + String.format("%010d", i)));
+          m.putDelete(new Text("colf"), colq);
+          bw.addMutation(m);
+        }
+      }
+
+      c.tableOperations().flush(tableName, null, null, true);
+
+      // compact the first 2 tablets so the new files with the deletes are gone
+      // so we can test that the data does not come back when the 3rd tablet is
+      // merged back with the other tablets as it still contains the original file
+      c.tableOperations().compact(tableName, new CompactionConfig().setStartRow(null)
+          .setEndRow(List.copyOf(splits).get(1)).setWait(true));
+      log.debug("Metadata after deleting rows 101 - 200 and 301 - 400");
+      printAndVerifyFileMetadata(getServerContext(), tableId, 4);
+
+      // Merge and print results
+      c.tableOperations().merge(tableName, null, null);
+      log.debug("Metadata after Merge");
+      printAndVerifyFileMetadata(getServerContext(), tableId, 4);
+
+      // Verify that the deleted rows can't be read after merge
+      verify(c, 100, 1, tableName);
+      verifyNoRows(c, 100, 101, tableName);
+      verify(c, 100, 201, tableName);
+      verifyNoRows(c, 100, 301, tableName);
+      verify(c, 600, 401, tableName);
+
+      // Verify that the MERGED marker was cleared
+      verifyMergedMarkerCleared(getServerContext(), tableId);
+    }
+  }
+
+  @Test
+  public void noChopMergeDeleteAcrossTablets() throws Exception {
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      createTableAndDisableCompactions(c, tableName, new NewTableConfiguration());
+      // disable compactions
+      c.tableOperations().setProperty(tableName, Property.TABLE_MAJC_RATIO.getKey(), "9999");
+      final TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      // First write 1000 rows to a file in the default tablet
+      ingest(c, 1000, 1, tableName);
+      c.tableOperations().flush(tableName, null, null, true);
+
+      log.debug("Metadata after Ingest");
+      printAndVerifyFileMetadata(getServerContext(), tableId, 1);
+
+      // Add splits so we end up with 10 tablets
+      final SortedSet<Text> splits = new TreeSet<>();
+      for (int i = 100; i <= 900; i += 100) {
+        splits.add(new Text("row_" + String.format("%010d", i)));
+      }
+      c.tableOperations().addSplits(tableName, splits);
+
+      log.debug("Metadata after Split");
+      verify(c, 1000, 1, tableName);
+      printAndVerifyFileMetadata(getServerContext(), tableId, 10);
+
+      // Go through and delete three blocks of rows
+      // 151 - 250, 451 - 550, 751 - 850
+      try (BatchWriter bw = c.createBatchWriter(tableName)) {
+        byte[] COL_PREFIX = "col_".getBytes(UTF_8);
+        Text colq = new Text(FastFormat.toZeroPaddedString(0, 7, 10, COL_PREFIX));
+
+        for (int j = 0; j <= 2; j++) {
+          for (int i = 151; i <= 250; i++) {
+            Mutation m = new Mutation(new Text("row_" + String.format("%010d", i + (j * 300))));
+            m.putDelete(new Text("colf"), colq);
+            bw.addMutation(m);
+          }
+        }
+      }
+
+      c.tableOperations().flush(tableName, null, null, true);
+
+      log.debug("Metadata after deleting rows 151 - 250, 451 - 550, 751 - 850");
+      // compact some of the tablets with deletes so we can test that the data does not come back
+      c.tableOperations().compact(tableName,
+          new CompactionConfig().setStartRow(new Text("row_" + String.format("%010d", 150)))
+              .setEndRow(new Text("row_" + String.format("%010d", 250))).setWait(true));
+      c.tableOperations().compact(tableName,
+          new CompactionConfig().setStartRow(new Text("row_" + String.format("%010d", 750)))
+              .setEndRow(new Text("row_" + String.format("%010d", 850))).setWait(true));
+      // Should be 16 files (10 for the original splits plus 2 extra files per deletion range across
+      // tablets)
+      printAndVerifyFileMetadata(getServerContext(), tableId, 12);
+
+      c.tableOperations().merge(tableName, null, null);
+      log.debug("Metadata after Merge");
+      printAndVerifyFileMetadata(getServerContext(), tableId, 12);
+      // Verify that the MERGED marker was cleared
+      verifyMergedMarkerCleared(getServerContext(), tableId);
+
+      // Verify that the deleted rows can't be read after merge
+      verify(c, 150, 1, tableName);
+      verifyNoRows(c, 100, 151, tableName);
+      verify(c, 200, 251, tableName);
+      verifyNoRows(c, 100, 451, tableName);
+      verify(c, 200, 551, tableName);
+      verifyNoRows(c, 100, 751, tableName);
+      verify(c, 150, 851, tableName);
+
+      // Compact and verify we clean up all the files and only 1 left
+      // Verify only 700 entries
+      c.tableOperations().compact(tableName, new CompactionConfig().setWait(true));
+      log.debug("Metadata after compact");
+      // Should just be 1 file with infinite range
+      Map<StoredTabletFile,DataFileValue> files =
+          printAndVerifyFileMetadata(getServerContext(), tableId, 1);
+      assertEquals(new Range(), files.keySet().stream().findFirst().orElseThrow().getRange());
+      assertEquals(700, files.values().stream().findFirst().orElseThrow().getNumEntries());
+    }
+  }
+
+  // Multiple splits/deletes/merges to show ranges work and carry forward
+  // Testing that we can split -> delete, merge, split -> delete, merge
+  // with deletions across boundaries
+  @Test
+  public void noChopMergeDeleteAcrossTabletsMultiple() throws Exception {
+    // Run initial test to populate table and merge which adds ranges to files
+    noChopMergeDeleteAcrossTablets();
+
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      final TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      log.debug("Metadata after initial test run");
+      printAndVerifyFileMetadata(getServerContext(), tableId, -1);
+
+      // Add splits so we end up with 10 tablets
+      final SortedSet<Text> splits = new TreeSet<>();
+      for (int i = 100; i <= 900; i += 100) {
+        splits.add(new Text("row_" + String.format("%010d", i)));
+      }
+      c.tableOperations().addSplits(tableName, splits);
+
+      log.debug("Metadata after Split for second time");
+      // Verify that the deleted rows can't be read after merge
+      verify(c, 150, 1, tableName);
+      verifyNoRows(c, 100, 151, tableName);
+      verify(c, 200, 251, tableName);
+      verifyNoRows(c, 100, 451, tableName);
+      verify(c, 200, 551, tableName);
+      verifyNoRows(c, 100, 751, tableName);
+      verify(c, 150, 851, tableName);
+      printAndVerifyFileMetadata(getServerContext(), tableId, -1);
+
+      c.tableOperations().flush(tableName, null, null, true);
+
+      // Go through and also delete 651 - 700
+      try (BatchWriter bw = c.createBatchWriter(tableName)) {
+        byte[] COL_PREFIX = "col_".getBytes(UTF_8);
+        Text colq = new Text(FastFormat.toZeroPaddedString(0, 7, 10, COL_PREFIX));
+
+        for (int i = 651; i <= 700; i++) {
+          Mutation m = new Mutation(new Text("row_" + String.format("%010d", i)));
+          m.putDelete(new Text("colf"), colq);
+          bw.addMutation(m);
+        }
+      }
+
+      c.tableOperations().flush(tableName, null, null, true);
+
+      log.debug("Metadata after deleting rows 151 - 250, 451 - 550, 651 - 700, 751 - 850");
+      // compact some of the tablets with deletes so we can test that the data does not come back
+      c.tableOperations().compact(tableName,
+          new CompactionConfig().setStartRow(new Text("row_" + String.format("%010d", 150)))
+              .setEndRow(new Text("row_" + String.format("%010d", 700))).setWait(true));
+
+      // Re-merge a second time after deleting more rows
+      c.tableOperations().merge(tableName, null, null);
+      log.debug("Metadata after second Merge");
+      printAndVerifyFileMetadata(getServerContext(), tableId, -1);
+      // Verify that the MERGED marker was cleared
+      verifyMergedMarkerCleared(getServerContext(), tableId);
+
+      // Verify that the deleted rows can't be read after merge
+      verify(c, 150, 1, tableName);
+      verifyNoRows(c, 100, 151, tableName);
+      verify(c, 200, 251, tableName);
+      verifyNoRows(c, 100, 451, tableName);
+      verify(c, 100, 551, tableName);
+      verifyNoRows(c, 50, 651, tableName);
+      verify(c, 50, 701, tableName);
+      verifyNoRows(c, 100, 751, tableName);
+      verify(c, 150, 851, tableName);
+    }
+  }
+
+  // Tests that after we merge and fence files, we can split and then
+  // merge a second time the same table which shows splits/merges work
+  // for files that already have ranges
+  @Test
+  public void noChopMergeTestMultipleMerges() throws Exception {
+    // Do initial merge which will fence off files
+    noChopMergeTest();
+
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+      final TableId tableId = TableId.of(c.tableOperations().tableIdMap().get(tableName));
+
+      log.debug("Metadata after initial no chop merge test");
+      printAndVerifyFileMetadata(getServerContext(), tableId, 4);
+
+      // Add splits so we end up with 4 tablets
+      final SortedSet<Text> splits = new TreeSet<>();
+      for (int i = 250; i <= 750; i += 250) {
+        splits.add(new Text("row_" + String.format("%010d", i)));
+      }
+      c.tableOperations().addSplits(tableName, splits);
+
+      log.debug("Metadata after Split");
+      // Verify after splitting for the second time
+      verify(c, 100, 1, tableName);
+      verifyNoRows(c, 100, 101, tableName);
+      verify(c, 100, 201, tableName);
+      verifyNoRows(c, 100, 301, tableName);
+      verify(c, 600, 401, tableName);
+      printAndVerifyFileMetadata(getServerContext(), tableId, -1);
+
+      // Re-Merge and print results. This tests merging with files
+      // that already have a range
+      c.tableOperations().merge(tableName, null, null);
+      log.debug("Metadata after Merge");
+      printAndVerifyFileMetadata(getServerContext(), tableId, -1);
+      // Verify that the MERGED marker was cleared
+      verifyMergedMarkerCleared(getServerContext(), tableId);
+
+      // Verify that the deleted rows can't be read after merge
+      verify(c, 100, 1, tableName);
+      verifyNoRows(c, 100, 101, tableName);
+      verify(c, 100, 201, tableName);
+      verifyNoRows(c, 100, 301, tableName);
+      verify(c, 600, 401, tableName);
+    }
+  }
+
+  private static void createTableAndDisableCompactions(AccumuloClient c, String tableName,
+      NewTableConfiguration ntc) throws Exception {
+    // disable compactions
+    ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "9999"));
+    c.tableOperations().create(tableName, ntc);
+  }
+
+  public static void ingest(AccumuloClient accumuloClient, int rows, int offset, String tableName)
+      throws Exception {
+    IngestParams params = new IngestParams(accumuloClient.properties(), tableName, rows);
+    params.cols = 1;
+    params.dataSize = 10;
+    params.startRow = offset;
+    params.columnFamily = "colf";
+    params.createTable = true;
+    TestIngest.ingest(accumuloClient, params);
+  }
+
+  private static void verify(AccumuloClient accumuloClient, int rows, int offset, String tableName)
+      throws Exception {
+    VerifyParams params = new VerifyParams(accumuloClient.properties(), tableName, rows);
+    params.rows = rows;
+    params.dataSize = 10;
+    params.startRow = offset;
+    params.columnFamily = "colf";
+    params.cols = 1;
+    VerifyIngest.verifyIngest(accumuloClient, params);
+  }
+
+  private static void verifyNoRows(AccumuloClient accumuloClient, int rows, int offset,
+      String tableName) throws Exception {
+    try {
+      verify(accumuloClient, rows, offset, tableName);
+      fail("Should have failed");
+    } catch (AccumuloException e) {
+      assertTrue(e.getMessage().contains("Did not read expected number of rows. Saw 0"));
+    }
+  }
+
   private String[] toStrings(Collection<Text> listSplits) {
     String[] result = new String[listSplits.size()];
     int i = 0;
@@ -181,7 +519,7 @@
 
   private void runMergeTest(AccumuloClient client, String table, String[] splits,
       String[] expectedSplits, String[] inserts, String start, String end) throws Exception {
-    System.out.println(
+    log.debug(
         "Running merge test " + table + " " + Arrays.asList(splits) + " " + start + " " + end);
 
     SortedSet<Text> splitSet = splits(splits);
@@ -190,7 +528,7 @@
     if (!splitSet.isEmpty()) {
       ntc = ntc.withSplits(splitSet);
     }
-    client.tableOperations().create(table, ntc);
+    createTableAndDisableCompactions(client, table, ntc);
 
     HashSet<String> expected = new HashSet<>();
     try (BatchWriter bw = client.createBatchWriter(table)) {
@@ -202,9 +540,20 @@
       }
     }
 
+    log.debug("Before Merge");
+    client.tableOperations().flush(table, null, null, true);
+    TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(table));
+    printAndVerifyFileMetadata(getServerContext(), tableId);
+
     client.tableOperations().merge(table, start == null ? null : new Text(start),
         end == null ? null : new Text(end));
 
+    client.tableOperations().flush(table, null, null, true);
+    log.debug("After Merge");
+    printAndVerifyFileMetadata(getServerContext(), tableId);
+    // Verify that the MERGED marker was cleared
+    verifyMergedMarkerCleared(getServerContext(), tableId);
+
     try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
 
       HashSet<String> observed = new HashSet<>();
@@ -250,10 +599,11 @@
           var tablet = tabletsMutator.mutateTablet(extent);
           ExternalCompactionId ecid = ExternalCompactionId.generate(UUID.randomUUID());
 
-          TabletFile tmpFile = new TabletFile(new Path("file:///accumulo/tables/t-0/b-0/c1.rf"));
+          ReferencedTabletFile tmpFile =
+              ReferencedTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/c1.rf"));
           CompactionExecutorId ceid = CompactionExecutorIdImpl.externalId("G1");
           Set<StoredTabletFile> jobFiles =
-              Set.of(new StoredTabletFile("file:///accumulo/tables/t-0/b-0/b2.rf"));
+              Set.of(StoredTabletFile.of(new Path("file:///accumulo/tables/t-0/b-0/b2.rf")));
           ExternalCompactionMetadata ecMeta = new ExternalCompactionMetadata(jobFiles, jobFiles,
               tmpFile, "localhost:4444", CompactionKind.SYSTEM, (short) 2, ceid, false, false, 44L);
           tablet.putExternalCompaction(ecid, ecMeta);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
index 3c65146..60ca6cb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataIT.java
@@ -22,7 +22,6 @@
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LAST;
 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.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -36,7 +35,7 @@
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -49,8 +48,7 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
-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;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
@@ -87,7 +85,8 @@
       // create a table to write some data to metadata table
       c.tableOperations().create(tableNames[0]);
 
-      try (Scanner rootScanner = c.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner rootScanner =
+          c.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY)) {
         rootScanner.setRange(TabletsSection.getRange());
         rootScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 
@@ -97,7 +96,7 @@
         }
 
         c.tableOperations().create(tableNames[1]);
-        c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+        c.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
 
         Set<String> files2 = new HashSet<>();
         for (Entry<Key,Value> entry : rootScanner) {
@@ -108,7 +107,7 @@
         assertTrue(!files2.isEmpty());
         assertNotEquals(files1, files2);
 
-        c.tableOperations().compact(MetadataTable.NAME, null, null, false, true);
+        c.tableOperations().compact(AccumuloTable.METADATA.tableName(), null, null, false, true);
 
         Set<String> files3 = new HashSet<>();
         for (Entry<Key,Value> entry : rootScanner) {
@@ -129,17 +128,17 @@
       for (String id : "1 2 3 4 5".split(" ")) {
         splits.add(new Text(id));
       }
-      c.tableOperations().addSplits(MetadataTable.NAME, splits);
+      c.tableOperations().addSplits(AccumuloTable.METADATA.tableName(), splits);
       for (String tableName : names) {
         c.tableOperations().create(tableName);
       }
-      c.tableOperations().merge(MetadataTable.NAME, null, null);
-      try (Scanner s = c.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
+      c.tableOperations().merge(AccumuloTable.METADATA.tableName(), null, null);
+      try (Scanner s = c.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY)) {
         s.setRange(DeletesSection.getRange());
         while (s.stream().findAny().isEmpty()) {
-          sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+          Thread.sleep(100);
         }
-        assertEquals(0, c.tableOperations().listSplits(MetadataTable.NAME).size());
+        assertEquals(0, c.tableOperations().listSplits(AccumuloTable.METADATA.tableName()).size());
       }
     }
   }
@@ -151,13 +150,13 @@
       c.tableOperations().create(tableName);
 
       // batch scan regular metadata table
-      try (BatchScanner s = c.createBatchScanner(MetadataTable.NAME)) {
+      try (BatchScanner s = c.createBatchScanner(AccumuloTable.METADATA.tableName())) {
         s.setRanges(Collections.singleton(new Range()));
         assertTrue(s.stream().anyMatch(Objects::nonNull));
       }
 
       // batch scan root metadata table
-      try (BatchScanner s = c.createBatchScanner(RootTable.NAME)) {
+      try (BatchScanner s = c.createBatchScanner(AccumuloTable.ROOT.tableName())) {
         s.setRanges(Collections.singleton(new Range()));
         assertTrue(s.stream().anyMatch(Objects::nonNull));
       }
@@ -168,7 +167,7 @@
   public void testAmpleReadTablets() throws Exception {
 
     try (ClientContext cc = (ClientContext) Accumulo.newClient().from(getClientProps()).build()) {
-      cc.securityOperations().grantTablePermission(cc.whoami(), MetadataTable.NAME,
+      cc.securityOperations().grantTablePermission(cc.whoami(), AccumuloTable.METADATA.tableName(),
           TablePermission.WRITE);
 
       SortedSet<Text> partitionKeys = new TreeSet<>();
@@ -182,12 +181,16 @@
       Text startRow = new Text("a");
       Text endRow = new Text("z");
 
-      // Call up Ample from the client context using table "t" and build
-      TabletsMetadata tablets = cc.getAmple().readTablets().forTable(TableId.of("1"))
-          .overlapping(startRow, endRow).fetch(FILES, LOCATION, LAST, PREV_ROW).build();
+      TabletMetadata tabletMetadata0;
+      TabletMetadata tabletMetadata1;
 
-      TabletMetadata tabletMetadata0 = tablets.stream().findFirst().orElseThrow();
-      TabletMetadata tabletMetadata1 = tablets.stream().skip(1).findFirst().orElseThrow();
+      // Call up Ample from the client context using table "t" and build
+      try (TabletsMetadata tm = cc.getAmple().readTablets().forTable(TableId.of("1"))
+          .overlapping(startRow, endRow).fetch(FILES, LOCATION, LAST, PREV_ROW).build()) {
+        var tablets = tm.stream().limit(2).collect(Collectors.toList());
+        tabletMetadata0 = tablets.get(0);
+        tabletMetadata1 = tablets.get(1);
+      }
 
       String infoTabletId0 = tabletMetadata0.getTableId().toString();
       String infoExtent0 = tabletMetadata0.getExtent().toString();
@@ -234,8 +237,10 @@
 
       // It is important here to use getTableProperties() and not getConfiguration()
       // because we want only the table properties and not a merged view
-      var rootTableProps = client.tableOperations().getTableProperties(RootTable.NAME);
-      var metadataTableProps = client.tableOperations().getTableProperties(MetadataTable.NAME);
+      var rootTableProps =
+          client.tableOperations().getTableProperties(AccumuloTable.ROOT.tableName());
+      var metadataTableProps =
+          client.tableOperations().getTableProperties(AccumuloTable.METADATA.tableName());
 
       // Verify root table config
       testCommonSystemTableConfig(rootTableProps);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
index f47d607..ae2900a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java
@@ -18,13 +18,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.time.Duration;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -32,10 +31,9 @@
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -66,18 +64,18 @@
       for (int i = 0; i < 1000; i++) {
         splits.add(new Text(String.format("%03d", i)));
       }
-      c.tableOperations().setProperty(MetadataTable.NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(),
-          "10000");
+      c.tableOperations().setProperty(AccumuloTable.METADATA.tableName(),
+          Property.TABLE_SPLIT_THRESHOLD.getKey(), "10000");
       // propagation time
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(5));
       for (int i = 0; i < 2; i++) {
         String tableName = "table" + i;
         log.info("Creating {} with splits", tableName);
         NewTableConfiguration ntc = new NewTableConfiguration().withSplits(splits);
         c.tableOperations().create(tableName, ntc);
         log.info("flushing");
-        c.tableOperations().flush(MetadataTable.NAME, null, null, true);
-        c.tableOperations().flush(RootTable.NAME, null, null, true);
+        c.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
+        c.tableOperations().flush(AccumuloTable.ROOT.tableName(), null, null, true);
       }
 
       while (true) {
@@ -97,7 +95,7 @@
         if (tablets == 2002) {
           break;
         }
-        sleepUninterruptibly(1, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(1));
       }
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
index 1853b40..4d29c20 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataSplitIT.java
@@ -18,18 +18,17 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.time.Duration;
 import java.util.Collections;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
@@ -49,15 +48,15 @@
   @Test
   public void test() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-      assertEquals(1, c.tableOperations().listSplits(MetadataTable.NAME).size());
-      c.tableOperations().setProperty(MetadataTable.NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(),
-          "500");
+      assertEquals(1, c.tableOperations().listSplits(AccumuloTable.METADATA.tableName()).size());
+      c.tableOperations().setProperty(AccumuloTable.METADATA.tableName(),
+          Property.TABLE_SPLIT_THRESHOLD.getKey(), "500");
       for (int i = 0; i < 10; i++) {
         c.tableOperations().create("table" + i);
-        c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+        c.tableOperations().flush(AccumuloTable.METADATA.tableName(), null, null, true);
       }
-      sleepUninterruptibly(10, TimeUnit.SECONDS);
-      assertTrue(c.tableOperations().listSplits(MetadataTable.NAME).size() > 2);
+      Thread.sleep(SECONDS.toMillis(10));
+      assertTrue(c.tableOperations().listSplits(AccumuloTable.METADATA.tableName()).size() > 2);
     }
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
index a588cb1..ec3b2eb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MonitorSslIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.io.File;
@@ -61,9 +62,9 @@
 
   @BeforeAll
   public static void initHttps() throws NoSuchAlgorithmException, KeyManagementException {
-    SSLContext ctx = SSLContext.getInstance("TLSv1.2");
+    SSLContext ctx = SSLContext.getInstance("TLSv1.3");
     TrustManager[] tm = {new TestTrustManager()};
-    ctx.init(new KeyManager[0], tm, random);
+    ctx.init(new KeyManager[0], tm, RANDOM.get());
     SSLContext.setDefault(ctx);
     HttpsURLConnection.setDefaultSSLSocketFactory(ctx.getSocketFactory());
     HttpsURLConnection.setDefaultHostnameVerifier(new TestHostnameVerifier());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java
index 95fef16..cd1b857 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/NativeMapIT.java
@@ -19,8 +19,8 @@
 package org.apache.accumulo.test.functional;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
-import static org.apache.accumulo.harness.AccumuloITBase.random;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -411,10 +411,10 @@
 
   // random length random field
   private static byte[] getRandomBytes(int maxLen) {
-    int len = random.nextInt(maxLen);
+    int len = RANDOM.get().nextInt(maxLen);
 
     byte[] f = new byte[len];
-    random.nextBytes(f);
+    RANDOM.get().nextBytes(f);
 
     return f;
   }
@@ -431,7 +431,7 @@
     for (int i = 0; i < 100000; i++) {
 
       Key k = new Key(getRandomBytes(97), getRandomBytes(13), getRandomBytes(31),
-          getRandomBytes(11), (random.nextLong() & 0x7fffffffffffffffL), false, false);
+          getRandomBytes(11), (RANDOM.get().nextLong() & 0x7fffffffffffffffL), false, false);
       Value v = new Value(getRandomBytes(511));
 
       testData.add(new Pair<>(k, v));
@@ -475,7 +475,7 @@
       System.out.println("test 11 nm mem " + nm.getMemoryUsed());
 
       // insert data again w/ different value
-      Collections.shuffle(testData, random);
+      Collections.shuffle(testData, RANDOM.get());
       // insert unsorted data
       for (Pair<Key,Value> pair : testData) {
         pair.getSecond().set(getRandomBytes(511));
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java b/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java
index 53306cf..43592da 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/PerTableCryptoIT.java
@@ -182,7 +182,7 @@
           try (PrintStream newOut = new PrintStream(baos)) {
             System.setOut(newOut);
             List<String> args = new ArrayList<>();
-            args.add(f.getPathStr());
+            args.add(f.getNormalizedPathStr());
             args.add("--props");
             args.add(getCluster().getAccumuloPropertiesPath());
             if (getClusterType() == ClusterType.STANDALONE && saslEnabled()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
index c36becd..8f89653 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/PermissionsIT.java
@@ -51,7 +51,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -693,8 +693,8 @@
 
         // check for read-only access to metadata table
         loginAs(rootUser);
-        verifyHasOnlyTheseTablePermissions(c, c.whoami(), MetadataTable.NAME, TablePermission.READ,
-            TablePermission.ALTER_TABLE);
+        verifyHasOnlyTheseTablePermissions(c, c.whoami(), AccumuloTable.METADATA.tableName(),
+            TablePermission.READ, TablePermission.ALTER_TABLE);
         String tableName = getUniqueNames(1)[0] + "__TABLE_PERMISSION_TEST__";
 
         // test each permission
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index 44eb5b2..27868ad 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -18,32 +18,16 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.harness.AccumuloITBase.STANDALONE_CAPABLE_CLUSTER;
 import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
-import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
 import java.net.URL;
 import java.security.cert.X509Certificate;
 import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.net.ssl.HostnameVerifier;
@@ -59,35 +43,23 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-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.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
-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.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.file.rfile.PrintInfo;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
 import org.apache.accumulo.core.util.MonitorUtil;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestIngest.IngestParams;
-import org.apache.accumulo.test.TestMultiTableIngest;
 import org.apache.accumulo.test.VerifyIngest;
 import org.apache.accumulo.test.VerifyIngest.VerifyParams;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
@@ -116,14 +88,6 @@
   static final int COLS = 1;
   static final String COLF = "colf";
 
-  @Test
-  public void invalidInstanceName() {
-    try (var client = Accumulo.newClient().to("fake_instance_name", cluster.getZooKeepers())
-        .as(getAdminPrincipal(), getAdminToken()).build()) {
-      assertThrows(RuntimeException.class, () -> client.instanceOperations().getTabletServers());
-    }
-  }
-
   @SuppressFBWarnings(value = {"PATH_TRAVERSAL_IN", "URLCONNECTION_SSRF_FD"},
       justification = "path provided by test; url provided by test")
   @Test
@@ -150,9 +114,9 @@
         if (monitorSslKeystore != null && !monitorSslKeystore.isEmpty()) {
           log.info(
               "Using HTTPS since monitor ssl keystore configuration was observed in accumulo configuration");
-          SSLContext ctx = SSLContext.getInstance("TLSv1.2");
+          var ctx = SSLContext.getInstance(Property.RPC_SSL_CLIENT_PROTOCOL.getDefaultValue());
           TrustManager[] tm = {new TestTrustManager()};
-          ctx.init(new KeyManager[0], tm, random);
+          ctx.init(new KeyManager[0], tm, RANDOM.get());
           SSLContext.setDefault(ctx);
           HttpsURLConnection.setDefaultSSLSocketFactory(ctx.getSocketFactory());
           HttpsURLConnection.setDefaultHostnameVerifier(new TestHostnameVerifier());
@@ -169,14 +133,14 @@
       var zLockPath =
           ServiceLock.path(ZooUtil.getRoot(accumuloClient.instanceOperations().getInstanceId())
               + Constants.ZMANAGER_LOCK);
-      byte[] managerLockData;
+      Optional<ServiceLockData> managerLockData;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
       control.stopAllServers(ServerType.MANAGER);
       control.stopAllServers(ServerType.TABLET_SERVER);
       control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
@@ -214,7 +178,7 @@
     verify(accumuloClient, rows, cols, width, offset, COLF, tableName);
   }
 
-  private static void verify(AccumuloClient accumuloClient, int rows, int cols, int width,
+  public static void verify(AccumuloClient accumuloClient, int rows, int cols, int width,
       int offset, String colf, String tableName) throws Exception {
     VerifyParams params = new VerifyParams(accumuloClient.properties(), tableName, rows);
     params.rows = rows;
@@ -230,38 +194,6 @@
   }
 
   @Test
-  public void multiTableTest() throws Exception {
-    // Write to multiple tables
-    final ClusterControl control = cluster.getClusterControl();
-    final String prefix = getClass().getSimpleName() + "_" + testName();
-    ExecutorService svc = Executors.newFixedThreadPool(2);
-    Future<Integer> p1 = svc.submit(() -> {
-      try {
-        return control.exec(TestMultiTableIngest.class, args("--count", Integer.toString(ROWS),
-            "-c", cluster.getClientPropsPath(), "--tablePrefix", prefix));
-      } catch (IOException e) {
-        log.error("Error running MultiTableIngest", e);
-        return -1;
-      }
-    });
-    Future<Integer> p2 = svc.submit(() -> {
-      try {
-        return control.exec(TestMultiTableIngest.class, args("--count", Integer.toString(ROWS),
-            "--readonly", "-c", cluster.getClientPropsPath(), "--tablePrefix", prefix));
-      } catch (IOException e) {
-        log.error("Error running MultiTableIngest", e);
-        return -1;
-      }
-    });
-    svc.shutdown();
-    while (!svc.isTerminated()) {
-      svc.awaitTermination(15, TimeUnit.SECONDS);
-    }
-    assertEquals(0, p1.get().intValue());
-    assertEquals(0, p2.get().intValue());
-  }
-
-  @Test
   public void largeTest() throws Exception {
     // write a few large values
     try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
@@ -313,159 +245,6 @@
     return m;
   }
 
-  @Test
-  public void localityGroupPerf() throws Exception {
-    // verify that locality groups can make look-ups faster
-    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
-      final String tableName = getUniqueNames(1)[0];
-      accumuloClient.tableOperations().create(tableName);
-      accumuloClient.tableOperations().setProperty(tableName, "table.group.g1", "colf");
-      accumuloClient.tableOperations().setProperty(tableName, "table.groups.enabled", "g1");
-      ingest(accumuloClient, 2000, 1, 50, 0, tableName);
-      accumuloClient.tableOperations().compact(tableName, null, null, true, true);
-      try (BatchWriter bw = accumuloClient.createBatchWriter(tableName)) {
-        bw.addMutation(m("zzzzzzzzzzz", "colf2", "cq", "value"));
-      }
-      long now = System.currentTimeMillis();
-      try (Scanner scanner = accumuloClient.createScanner(tableName, Authorizations.EMPTY)) {
-        scanner.fetchColumnFamily(new Text("colf"));
-        scanner.forEach((k, v) -> {});
-      }
-      long diff = System.currentTimeMillis() - now;
-      now = System.currentTimeMillis();
-
-      try (Scanner scanner = accumuloClient.createScanner(tableName, Authorizations.EMPTY)) {
-        scanner.fetchColumnFamily(new Text("colf2"));
-        scanner.forEach((k, v) -> {});
-      }
-      long diff2 = System.currentTimeMillis() - now;
-      assertTrue(diff2 < diff);
-    }
-  }
-
-  /**
-   * create a locality group, write to it and ensure it exists in the RFiles that result
-   */
-  @Test
-  public void sunnyLG() throws Exception {
-    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
-      final String tableName = getUniqueNames(1)[0];
-      accumuloClient.tableOperations().create(tableName);
-      Map<String,Set<Text>> groups = new TreeMap<>();
-      groups.put("g1", Collections.singleton(t("colf")));
-      accumuloClient.tableOperations().setLocalityGroups(tableName, groups);
-      verifyLocalityGroupsInRFile(accumuloClient, tableName);
-    }
-  }
-
-  /**
-   * Pretty much identical to sunnyLG, but verifies locality groups are created when configured in
-   * NewTableConfiguration prior to table creation.
-   */
-  @Test
-  public void sunnyLGUsingNewTableConfiguration() throws Exception {
-    // create a locality group, write to it and ensure it exists in the RFiles that result
-    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
-      final String tableName = getUniqueNames(1)[0];
-      NewTableConfiguration ntc = new NewTableConfiguration();
-      Map<String,Set<Text>> groups = new HashMap<>();
-      groups.put("g1", Collections.singleton(t("colf")));
-      ntc.setLocalityGroups(groups);
-      accumuloClient.tableOperations().create(tableName, ntc);
-      verifyLocalityGroupsInRFile(accumuloClient, tableName);
-    }
-  }
-
-  private void verifyLocalityGroupsInRFile(final AccumuloClient accumuloClient,
-      final String tableName) throws Exception {
-    ingest(accumuloClient, 2000, 1, 50, 0, tableName);
-    verify(accumuloClient, 2000, 1, 50, 0, tableName);
-    accumuloClient.tableOperations().flush(tableName, null, null, true);
-    try (BatchScanner bscanner =
-        accumuloClient.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 1)) {
-      String tableId = accumuloClient.tableOperations().tableIdMap().get(tableName);
-      bscanner.setRanges(
-          Collections.singletonList(new Range(new Text(tableId + ";"), new Text(tableId + "<"))));
-      bscanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      boolean foundFile = false;
-      for (Entry<Key,Value> entry : bscanner) {
-        foundFile = true;
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        PrintStream oldOut = System.out;
-        try (PrintStream newOut = new PrintStream(baos)) {
-          System.setOut(newOut);
-          List<String> args = new ArrayList<>();
-          args.add(entry.getKey().getColumnQualifier().toString());
-          args.add("--props");
-          args.add(getCluster().getAccumuloPropertiesPath());
-          if (getClusterType() == ClusterType.STANDALONE && saslEnabled()) {
-            args.add("--config");
-            StandaloneAccumuloCluster sac = (StandaloneAccumuloCluster) cluster;
-            String hadoopConfDir = sac.getHadoopConfDir();
-            args.add(new Path(hadoopConfDir, "core-site.xml").toString());
-            args.add(new Path(hadoopConfDir, "hdfs-site.xml").toString());
-          }
-          log.info("Invoking PrintInfo with {}", args);
-          PrintInfo.main(args.toArray(new String[args.size()]));
-          newOut.flush();
-          String stdout = baos.toString();
-          assertTrue(stdout.contains("Locality group           : g1"));
-          assertTrue(stdout.contains("families        : [colf]"));
-        } finally {
-          System.setOut(oldOut);
-        }
-      }
-      assertTrue(foundFile);
-    }
-  }
-
-  @Test
-  public void localityGroupChange() throws Exception {
-    // Make changes to locality groups and ensure nothing is lost
-    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
-      String table = getUniqueNames(1)[0];
-      TableOperations to = accumuloClient.tableOperations();
-      to.create(table);
-      String[] config = {"lg1:colf", null, "lg1:colf,xyz", "lg1:colf,xyz;lg2:c1,c2"};
-      int i = 0;
-      for (String cfg : config) {
-        to.setLocalityGroups(table, getGroups(cfg));
-        ingest(accumuloClient, ROWS * (i + 1), 1, 50, ROWS * i, table);
-        to.flush(table, null, null, true);
-        verify(accumuloClient, 0, 1, 50, ROWS * (i + 1), table);
-        i++;
-      }
-      to.delete(table);
-      to.create(table);
-      config = new String[] {"lg1:colf", null, "lg1:colf,xyz", "lg1:colf;lg2:colf",};
-      i = 1;
-      for (String cfg : config) {
-        ingest(accumuloClient, ROWS * i, 1, 50, 0, table);
-        ingest(accumuloClient, ROWS * i, 1, 50, 0, "xyz", table);
-        to.setLocalityGroups(table, getGroups(cfg));
-        to.flush(table, null, null, true);
-        verify(accumuloClient, ROWS * i, 1, 50, 0, table);
-        verify(accumuloClient, ROWS * i, 1, 50, 0, "xyz", table);
-        i++;
-      }
-    }
-  }
-
-  private Map<String,Set<Text>> getGroups(String cfg) {
-    Map<String,Set<Text>> groups = new TreeMap<>();
-    if (cfg != null) {
-      for (String group : cfg.split(";")) {
-        String[] parts = group.split(":");
-        Set<Text> cols = new HashSet<>();
-        for (String col : parts[1].split(",")) {
-          cols.add(t(col));
-        }
-        groups.put(parts[1], cols);
-      }
-    }
-    return groups;
-  }
-
   @SuppressFBWarnings(value = "WEAK_TRUST_MANAGER",
       justification = "trust manager is okay for testing")
   private static class TestTrustManager implements X509TrustManager {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java
index 7291a42..c65cd0a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RecoveryWithEmptyRFileIT.java
@@ -31,11 +31,12 @@
 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.file.rfile.CreateEmpty;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.tserver.util.CreateEmpty;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -83,16 +84,18 @@
       client.tableOperations().offline(tableName, true);
 
       log.debug("Replacing rfile(s) with empty");
-      try (Scanner meta = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner meta =
+          client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         String tableId = client.tableOperations().tableIdMap().get(tableName);
         meta.setRange(new Range(new Text(tableId + ";"), new Text(tableId + "<")));
         meta.fetchColumnFamily(DataFileColumnFamily.NAME);
         boolean foundFile = false;
         for (Entry<Key,Value> entry : meta) {
           foundFile = true;
-          Path rfile = new Path(entry.getKey().getColumnQualifier().toString());
+          Path rfile = StoredTabletFile.of(entry.getKey().getColumnQualifier()).getPath();
           log.debug("Removing rfile '{}'", rfile);
           cluster.getFileSystem().delete(rfile, false);
+          // following depends on create-empty defaults to rfile.
           Process processInfo = cluster.exec(CreateEmpty.class, rfile.toString()).getProcess();
           assertEquals(0, processInfo.waitFor());
         }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java
index b7194b8..42078df 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java
@@ -36,7 +36,7 @@
 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.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
@@ -180,7 +180,8 @@
 
   private Table<String,String,MutableInt> getCounts(AccumuloClient client, String tablename)
       throws TableNotFoundException {
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+    try (Scanner s =
+        client.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
       s.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
       TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tablename));
       s.setRange(TabletsSection.getRange(tableId));
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
index 3653217..41dbb78 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -18,16 +18,16 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 import java.io.IOException;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterControl;
 import org.apache.accumulo.core.Constants;
@@ -35,10 +35,11 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -83,7 +84,7 @@
       svc.shutdown();
     }
 
-    while (!svc.awaitTermination(10, TimeUnit.SECONDS)) {
+    while (!svc.awaitTermination(10, SECONDS)) {
       log.info("Waiting for threadpool to terminate");
     }
   }
@@ -133,27 +134,27 @@
       ZooCache zcache = cluster.getServerContext().getZooCache();
       var zLockPath = ServiceLock
           .path(ZooUtil.getRoot(c.instanceOperations().getInstanceId()) + Constants.ZMANAGER_LOCK);
-      byte[] managerLockData;
+      Optional<ServiceLockData> managerLockData;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
 
       cluster.start();
-      sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
+      Thread.sleep(5);
       control.stopAllServers(ServerType.MANAGER);
 
-      managerLockData = new byte[0];
+      managerLockData = null;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
       cluster.start();
       VerifyIngest.verifyIngest(c, params);
     }
@@ -184,14 +185,14 @@
       ZooCache zcache = cluster.getServerContext().getZooCache();
       var zLockPath = ServiceLock
           .path(ZooUtil.getRoot(c.instanceOperations().getInstanceId()) + Constants.ZMANAGER_LOCK);
-      byte[] managerLockData;
+      Optional<ServiceLockData> managerLockData;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
 
       cluster.start();
       assertEquals(0, ret.get().intValue());
@@ -263,8 +264,8 @@
       }
       assertNotNull(splitThreshold);
       try {
-        c.tableOperations().setProperty(MetadataTable.NAME, Property.TABLE_SPLIT_THRESHOLD.getKey(),
-            "20K");
+        c.tableOperations().setProperty(AccumuloTable.METADATA.tableName(),
+            Property.TABLE_SPLIT_THRESHOLD.getKey(), "20K");
         TestIngest.ingest(c, params);
         c.tableOperations().flush(tableName, null, null, false);
         VerifyIngest.verifyIngest(c, params);
@@ -272,7 +273,7 @@
       } finally {
         if (getClusterType() == ClusterType.STANDALONE) {
           getCluster().start();
-          c.tableOperations().setProperty(MetadataTable.NAME,
+          c.tableOperations().setProperty(AccumuloTable.METADATA.tableName(),
               Property.TABLE_SPLIT_THRESHOLD.getKey(), splitThreshold);
         }
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
index 19224c4..1915324 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -18,14 +18,13 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.cluster.ClusterControl;
 import org.apache.accumulo.core.client.Accumulo;
@@ -77,7 +76,7 @@
       svc.shutdown();
     }
 
-    while (!svc.awaitTermination(10, TimeUnit.SECONDS)) {
+    while (!svc.awaitTermination(10, SECONDS)) {
       log.info("Waiting for threadpool to terminate");
     }
   }
@@ -103,7 +102,7 @@
       });
 
       for (int i = 0; i < 2; i++) {
-        sleepUninterruptibly(10, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(10));
         control.stopAllServers(ServerType.TABLET_SERVER);
         control.startAllServers(ServerType.TABLET_SERVER);
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java
index c880052..5c4cd01 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanIdIT.java
@@ -18,7 +18,8 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -37,7 +38,6 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -136,7 +136,7 @@
 
         if (resultsByWorker.size() < NUM_SCANNERS) {
           log.trace("Results reported {}", resultsByWorker.size());
-          sleepUninterruptibly(750, TimeUnit.MILLISECONDS);
+          Thread.sleep(750);
         } else {
           // each worker has reported at least one result.
           testInProgress.set(false);
@@ -144,7 +144,7 @@
           log.debug("Final result count {}", resultsByWorker.size());
 
           // delay to allow scanners to react to end of test and cleanly close.
-          sleepUninterruptibly(1, TimeUnit.SECONDS);
+          Thread.sleep(SECONDS.toMillis(1));
         }
 
       }
@@ -306,7 +306,8 @@
    *
    * @param client Accumulo client to test cluster or MAC instance.
    */
-  private void addSplits(final AccumuloClient client, final String tableName) {
+  private void addSplits(final AccumuloClient client, final String tableName)
+      throws InterruptedException {
 
     SortedSet<Text> splits = createSplits();
 
@@ -316,7 +317,8 @@
 
       client.tableOperations().offline(tableName, true);
 
-      sleepUninterruptibly(2, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(2));
+
       client.tableOperations().online(tableName, true);
 
       for (Text split : client.tableOperations().listSplits(tableName)) {
@@ -363,7 +365,7 @@
 
       for (int i = 0; i < NUM_DATA_ROWS; i++) {
 
-        Text rowId = new Text(String.format("%d", ((random.nextInt(10) * 100) + i)));
+        Text rowId = new Text(String.format("%d", ((RANDOM.get().nextInt(10) * 100) + i)));
 
         Mutation m = new Mutation(rowId);
         m.put("fam1", "count", Integer.toString(i));
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
index 53fb54d..65ee4c8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanIteratorIT.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
@@ -50,12 +49,10 @@
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Tag(SUNNY_DAY)
 public class ScanIteratorIT extends AccumuloClusterHarness {
   private static final Logger log = LoggerFactory.getLogger(ScanIteratorIT.class);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScanSessionTimeOutIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScanSessionTimeOutIT.java
index 60d3603..511ac42 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ScanSessionTimeOutIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ScanSessionTimeOutIT.java
@@ -18,13 +18,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.time.Duration;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -120,7 +119,7 @@
         verify(iter, 0, 200);
 
         // sleep three times the session timeout
-        sleepUninterruptibly(9, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(9));
 
         verify(iter, 200, 100000);
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScannerContextIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScannerContextIT.java
index 607386b..5aa9b60 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ScannerContextIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ScannerContextIT.java
@@ -42,7 +42,6 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,13 +51,8 @@
 
 public class ScannerContextIT extends AccumuloClusterHarness {
 
-  private static final String CONTEXT = ScannerContextIT.class.getSimpleName();
-  @SuppressWarnings("removal")
-  private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
-      Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-  private static final String CONTEXT_PROPERTY = VFS_CONTEXT_CLASSPATH_PROPERTY + CONTEXT;
   private static final String CONTEXT_DIR = "file://" + System.getProperty("user.dir") + "/target";
-  private static final String CONTEXT_CLASSPATH = CONTEXT_DIR + "/Test.jar";
+  private static final String CONTEXT = CONTEXT_DIR + "/Test.jar";
   private static int ITERATIONS = 10;
   private static final long WAIT = 7000;
 
@@ -76,7 +70,7 @@
     fs = FileSystem.get(cluster.getServerContext().getHadoopConf());
   }
 
-  private Path copyTestIteratorsJarToTmp() throws IOException {
+  private Path copyTestIteratorsJarToTmp() throws IOException, InterruptedException {
     // Copy the test iterators jar to tmp
     Path baseDir = new Path(System.getProperty("user.dir"));
     Path targetDir = new Path(baseDir, "target");
@@ -84,7 +78,7 @@
     Path dstPath = new Path(CONTEXT_DIR + "/Test.jar");
     fs.copyFromLocalFile(jarPath, dstPath);
     // Sleep to ensure jar change gets picked up
-    UtilWaitThread.sleep(WAIT);
+    Thread.sleep(WAIT);
     return dstPath;
   }
 
@@ -92,8 +86,6 @@
   public void test() throws Exception {
     Path dstPath = copyTestIteratorsJarToTmp();
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      // Set the classloader context property on the table to point to the test iterators jar file.
-      c.instanceOperations().setProperty(CONTEXT_PROPERTY, CONTEXT_CLASSPATH);
 
       // Insert rows with the word "Test" in the value.
       String tableName = getUniqueNames(1)[0];
@@ -136,19 +128,18 @@
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       // Create two contexts FOO and ScanContextIT. The FOO context will point to a classpath
       // that contains nothing. The ScanContextIT context will point to the test iterators jar
-      String tableContext = "FOO";
-      String tableContextProperty = VFS_CONTEXT_CLASSPATH_PROPERTY + tableContext;
+      String tableContextProperty = Property.TABLE_CLASSLOADER_CONTEXT.getKey();
       String tableContextDir = "file://" + System.getProperty("user.dir") + "/target";
       String tableContextClasspath = tableContextDir + "/TestFoo.jar";
-      // Define both contexts
-      c.instanceOperations().setProperty(tableContextProperty, tableContextClasspath);
-      c.instanceOperations().setProperty(CONTEXT_PROPERTY, CONTEXT_CLASSPATH);
+
+      // Set the ScanContextIT context on the instance
+      c.instanceOperations().setProperty(tableContextProperty, CONTEXT);
 
       String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
       // Set the FOO context on the table
       c.tableOperations().setProperty(tableName, Property.TABLE_CLASSLOADER_CONTEXT.getKey(),
-          tableContext);
+          tableContextClasspath);
       try (BatchWriter bw = c.createBatchWriter(tableName)) {
         for (int i = 0; i < ITERATIONS; i++) {
           Mutation m = new Mutation("row" + i);
@@ -184,8 +175,6 @@
   public void testOneScannerDoesntInterfereWithAnother() throws Exception {
     Path dstPath = copyTestIteratorsJarToTmp();
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      // Set the classloader context property on the table to point to the test iterators jar file.
-      c.instanceOperations().setProperty(CONTEXT_PROPERTY, CONTEXT_CLASSPATH);
 
       // Insert rows with the word "Test" in the value.
       String tableName = getUniqueNames(1)[0];
@@ -230,8 +219,6 @@
   public void testClearContext() throws Exception {
     Path dstPath = copyTestIteratorsJarToTmp();
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      // Set the classloader context property on the table to point to the test iterators jar file.
-      c.instanceOperations().setProperty(CONTEXT_PROPERTY, CONTEXT_CLASSPATH);
 
       // Insert rows with the word "Test" in the value.
       String tableName = getUniqueNames(1)[0];
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ScannerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ScannerIT.java
index 805b791..697452c 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ScannerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ScannerIT.java
@@ -34,7 +34,6 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.junit.jupiter.api.Test;
 
@@ -80,7 +79,7 @@
           nanosWithWait += System.nanoTime() - startTime;
 
           // While we "do work" in the client, we should be fetching the next result
-          UtilWaitThread.sleep(100L);
+          Thread.sleep(100L);
           iterator.next();
           startTime = System.nanoTime();
         }
@@ -100,7 +99,7 @@
           nanosWithNoWait += System.nanoTime() - startTime;
 
           // While we "do work" in the client, we should be fetching the next result
-          UtilWaitThread.sleep(100L);
+          Thread.sleep(100L);
           iterator.next();
           startTime = System.nanoTime();
         }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
index 262ed6f..5b8f413 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java
@@ -18,14 +18,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.time.Duration;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -88,7 +86,7 @@
         }
       });
 
-      sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+      Thread.sleep(500);
 
       // should be able to scan now
       try (Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java
index a6267c0..60ab7dd 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java
@@ -18,14 +18,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
 
 import java.io.IOException;
 import java.time.Duration;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -48,7 +46,7 @@
   public void shutdownDuringIngest() throws Exception {
     Process ingest = cluster
         .exec(TestIngest.class, "-c", cluster.getClientPropsPath(), "--createTable").getProcess();
-    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+    Thread.sleep(100);
     assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     ingest.destroy();
   }
@@ -60,7 +58,7 @@
             .getProcess().waitFor());
     Process verify =
         cluster.exec(VerifyIngest.class, "-c", cluster.getClientPropsPath()).getProcess();
-    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+    Thread.sleep(100);
     assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     verify.destroy();
   }
@@ -72,7 +70,7 @@
             .getProcess().waitFor());
     Process deleter =
         cluster.exec(TestRandomDeletes.class, "-c", cluster.getClientPropsPath()).getProcess();
-    sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+    Thread.sleep(100);
     assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
     deleter.destroy();
   }
@@ -94,7 +92,7 @@
         }
       });
       async.start();
-      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
       assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor());
       if (ref.get() != null) {
         throw ref.get();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index 2332d4c..be01b92 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -18,14 +18,13 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -34,8 +33,8 @@
 import org.apache.accumulo.core.clientImpl.Credentials;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.minicluster.MemoryUnit;
@@ -72,7 +71,7 @@
       params.rows = 5000;
       TestIngest.ingest(c, params);
       c.tableOperations().flush("test_ingest", null, null, false);
-      sleepUninterruptibly(45, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(45));
       Credentials creds = new Credentials("root", new PasswordToken(ROOT_PASSWORD));
 
       int unassignedTablets = 1;
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SlowConstraint.java b/test/src/main/java/org/apache/accumulo/test/functional/SlowConstraint.java
index 3177d02..86eacf1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SlowConstraint.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SlowConstraint.java
@@ -18,10 +18,9 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.constraints.Constraint;
@@ -35,7 +34,12 @@
 
   @Override
   public List<Short> check(Environment env, Mutation mutation) {
-    sleepUninterruptibly(20, TimeUnit.SECONDS);
+    try {
+      Thread.sleep(SECONDS.toMillis(20));
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("interrupted during sleep", ex);
+    }
     return null;
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
index 714ace0..e727b90 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SlowIterator.java
@@ -18,12 +18,9 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -57,14 +54,24 @@
 
   @Override
   public void next() throws IOException {
-    sleepUninterruptibly(sleepTime, TimeUnit.MILLISECONDS);
+    try {
+      Thread.sleep(sleepTime);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IOException("interrupted during sleep", ex);
+    }
     super.next();
   }
 
   @Override
   public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive)
       throws IOException {
-    sleepUninterruptibly(seekSleepTime, TimeUnit.MILLISECONDS);
+    try {
+      Thread.sleep(seekSleepTime);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IOException("interrupted during sleep", ex);
+    }
     super.seek(range, columnFamilies, inclusive);
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
index 9db3365..8169037 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.test.functional;
 
 import static java.util.Collections.singletonMap;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assumptions.assumeTrue;
@@ -30,7 +30,6 @@
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -46,7 +45,7 @@
 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.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -146,10 +145,10 @@
       TestIngest.ingest(c, params);
       VerifyIngest.verifyIngest(c, params);
       while (c.tableOperations().listSplits(table).size() < 10) {
-        sleepUninterruptibly(15, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(15));
       }
       TableId id = TableId.of(c.tableOperations().tableIdMap().get(table));
-      try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         KeyExtent extent = new KeyExtent(id, null, null);
         s.setRange(extent.toMetaRange());
         TabletColumnFamily.PREV_ROW_COLUMN.fetch(s);
@@ -183,9 +182,9 @@
 
       c.tableOperations().create(tableName, new NewTableConfiguration().setProperties(props));
 
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(5));
       ReadWriteIT.interleaveTest(c, tableName);
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(5));
       int numSplits = c.tableOperations().listSplits(tableName).size();
       while (numSplits <= 20) {
         log.info("Waiting for splits to happen");
@@ -205,7 +204,7 @@
       DeleteIT.deleteTest(c, getCluster(), tableName);
       c.tableOperations().flush(tableName, null, null, true);
       for (int i = 0; i < 5; i++) {
-        sleepUninterruptibly(10, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(10));
         if (c.tableOperations().listSplits(tableName).size() > 20) {
           break;
         }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
index 7b5b59a..a9119a4 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -45,16 +44,18 @@
 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.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.file.rfile.RFile;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 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.Ample.TabletMutator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
@@ -113,7 +114,8 @@
       public void unableToMonitorLockNode(Exception e) {
         System.exit(-1);
       }
-    }, "foo".getBytes(UTF_8));
+    }, new ServiceLockData(UUID.randomUUID(), "foo", ThriftService.TSERV,
+        ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     if (!gotLock) {
       System.err.println("Failed to get lock " + zPath);
@@ -153,7 +155,7 @@
 
     Text midRow = new Text(mr);
 
-    SortedMap<StoredTabletFile,DataFileValue> splitMapFiles = null;
+    SortedMap<StoredTabletFile,DataFileValue> splitDataFiles = null;
 
     for (int i = 0; i < extents.length; i++) {
       KeyExtent extent = extents[i];
@@ -162,17 +164,17 @@
       String tdir =
           context.getTablesDirs().iterator().next() + "/" + extent.tableId() + "/" + dirName;
       MetadataTableUtil.addTablet(extent, dirName, context, TimeType.LOGICAL, zl);
-      SortedMap<TabletFile,DataFileValue> mapFiles = new TreeMap<>();
-      mapFiles.put(new TabletFile(new Path(tdir + "/" + RFile.EXTENSION + "_000_000")),
+      SortedMap<ReferencedTabletFile,DataFileValue> dataFiles = new TreeMap<>();
+      dataFiles.put(new ReferencedTabletFile(new Path(tdir + "/" + RFile.EXTENSION + "_000_000")),
           new DataFileValue(1000017 + i, 10000 + i));
 
       int tid = 0;
       TransactionWatcher.ZooArbitrator.start(context, Constants.BULK_ARBITRATOR_TYPE, tid);
       SortedMap<StoredTabletFile,DataFileValue> storedFiles =
-          new TreeMap<>(MetadataTableUtil.updateTabletDataFile(tid, extent, mapFiles,
+          new TreeMap<>(MetadataTableUtil.updateTabletDataFile(tid, extent, dataFiles,
               new MetadataTime(0, TimeType.LOGICAL), context, zl));
       if (i == extentToSplit) {
-        splitMapFiles = storedFiles;
+        splitDataFiles = storedFiles;
       }
     }
 
@@ -181,29 +183,29 @@
     KeyExtent high = new KeyExtent(extent.tableId(), extent.endRow(), midRow);
     KeyExtent low = new KeyExtent(extent.tableId(), midRow, extent.prevEndRow());
 
-    splitPartiallyAndRecover(context, extent, high, low, .4, splitMapFiles, midRow,
+    splitPartiallyAndRecover(context, extent, high, low, .4, splitDataFiles, midRow,
         "localhost:1234", failPoint, zl);
   }
 
-  private static Map<Long,List<TabletFile>> getBulkFilesLoaded(ServerContext context,
+  private static Map<Long,List<ReferencedTabletFile>> getBulkFilesLoaded(ServerContext context,
       KeyExtent extent) {
-    Map<Long,List<TabletFile>> bulkFiles = new HashMap<>();
+    Map<Long,List<ReferencedTabletFile>> bulkFiles = new HashMap<>();
 
-    context.getAmple().readTablet(extent).getLoaded()
-        .forEach((path, txid) -> bulkFiles.computeIfAbsent(txid, k -> new ArrayList<>()).add(path));
+    context.getAmple().readTablet(extent).getLoaded().forEach((path, txid) -> bulkFiles
+        .computeIfAbsent(txid, k -> new ArrayList<>()).add(path.getTabletFile()));
 
     return bulkFiles;
   }
 
   private void splitPartiallyAndRecover(ServerContext context, KeyExtent extent, KeyExtent high,
-      KeyExtent low, double splitRatio, SortedMap<StoredTabletFile,DataFileValue> mapFiles,
+      KeyExtent low, double splitRatio, SortedMap<StoredTabletFile,DataFileValue> dataFiles,
       Text midRow, String location, int steps, ServiceLock zl) throws Exception {
 
     SortedMap<StoredTabletFile,DataFileValue> lowDatafileSizes = new TreeMap<>();
     SortedMap<StoredTabletFile,DataFileValue> highDatafileSizes = new TreeMap<>();
     List<StoredTabletFile> highDatafilesToRemove = new ArrayList<>();
 
-    MetadataTableUtil.splitDatafiles(midRow, splitRatio, new HashMap<>(), mapFiles,
+    MetadataTableUtil.splitDatafiles(midRow, splitRatio, new HashMap<>(), dataFiles,
         lowDatafileSizes, highDatafileSizes, highDatafilesToRemove);
 
     MetadataTableUtil.splitTablet(high, extent.prevEndRow(), splitRatio, context, zl, Set.of());
@@ -215,7 +217,7 @@
     tabletMutator.mutate();
 
     if (steps >= 1) {
-      Map<Long,List<TabletFile>> bulkFiles = getBulkFilesLoaded(context, high);
+      Map<Long,List<ReferencedTabletFile>> bulkFiles = getBulkFilesLoaded(context, high);
 
       ManagerMetadataUtil.addNewTablet(context, low, "lowDir", instance, lowDatafileSizes,
           bulkFiles, new MetadataTime(0, TimeType.LOGICAL), -1L, -1L, zl);
@@ -236,8 +238,10 @@
       ensureTabletHasNoUnexpectedMetadataEntries(context, low, lowDatafileSizes);
       ensureTabletHasNoUnexpectedMetadataEntries(context, high, highDatafileSizes);
 
-      Map<Long,? extends Collection<TabletFile>> lowBulkFiles = getBulkFilesLoaded(context, low);
-      Map<Long,? extends Collection<TabletFile>> highBulkFiles = getBulkFilesLoaded(context, high);
+      Map<Long,? extends Collection<ReferencedTabletFile>> lowBulkFiles =
+          getBulkFilesLoaded(context, low);
+      Map<Long,? extends Collection<ReferencedTabletFile>> highBulkFiles =
+          getBulkFilesLoaded(context, high);
 
       if (!lowBulkFiles.equals(highBulkFiles)) {
         throw new Exception(" " + lowBulkFiles + " != " + highBulkFiles + " " + low + " " + high);
@@ -248,13 +252,14 @@
       }
     } else {
       assertEquals(extent, fixedExtent);
-      ensureTabletHasNoUnexpectedMetadataEntries(context, extent, mapFiles);
+      ensureTabletHasNoUnexpectedMetadataEntries(context, extent, dataFiles);
     }
   }
 
   private void ensureTabletHasNoUnexpectedMetadataEntries(ServerContext context, KeyExtent extent,
-      SortedMap<StoredTabletFile,DataFileValue> expectedMapFiles) throws Exception {
-    try (Scanner scanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) {
+      SortedMap<StoredTabletFile,DataFileValue> expectedDataFiles) throws Exception {
+    try (Scanner scanner =
+        new ScannerImpl(context, AccumuloTable.METADATA.tableId(), Authorizations.EMPTY)) {
       scanner.setRange(extent.toMetaRange());
 
       HashSet<ColumnFQ> expectedColumns = new HashSet<>();
@@ -279,8 +284,8 @@
         Key key = entry.getKey();
 
         if (!key.getRow().equals(extent.toMetaRow())) {
-          throw new Exception(
-              "Tablet " + extent + " contained unexpected " + MetadataTable.NAME + " entry " + key);
+          throw new Exception("Tablet " + extent + " contained unexpected "
+              + AccumuloTable.METADATA.tableName() + " entry " + key);
         }
 
         if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
@@ -298,8 +303,8 @@
           continue;
         }
 
-        throw new Exception(
-            "Tablet " + extent + " contained unexpected " + MetadataTable.NAME + " entry " + key);
+        throw new Exception("Tablet " + extent + " contained unexpected "
+            + AccumuloTable.METADATA.tableName() + " entry " + key);
       }
 
       if (expectedColumns.size() > 1 || (expectedColumns.size() == 1)) {
@@ -308,9 +313,9 @@
 
       assertTrue(sawPer);
 
-      SortedMap<StoredTabletFile,DataFileValue> fixedMapFiles =
+      SortedMap<StoredTabletFile,DataFileValue> fixedDataFiles =
           MetadataTableUtil.getFileAndLogEntries(context, extent).getSecond();
-      verifySame(expectedMapFiles, fixedMapFiles);
+      verifySame(expectedDataFiles, fixedDataFiles);
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
index 008237e..a63b35b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
@@ -83,19 +83,7 @@
     try (AccumuloClient client = Accumulo.newClient().from(props).build()) {
       BulkIT.runTest(client, cluster.getFileSystem(),
           new Path(getCluster().getConfig().getDir().getAbsolutePath(), "tmp"),
-          getUniqueNames(1)[0], this.getClass().getName(), testName(), true);
+          getUniqueNames(1)[0], this.getClass().getName(), testName());
     }
   }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  @Timeout(value = 1, unit = MINUTES)
-  public void mapReduce() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      // testing old mapreduce code from core jar; the new mapreduce module should have its own test
-      // case which checks functionality with ssl enabled
-      org.apache.accumulo.test.mapreduce.MapReduceIT.runTest(client, getCluster());
-    }
-  }
-
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java
index a6568a5..6e422eb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java
@@ -69,10 +69,4 @@
   public void bulk() throws Exception {
     super.bulk();
   }
-
-  @Override
-  @Test
-  public void mapReduce() throws Exception {
-    super.mapReduce();
-  }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
index e133097..cee7832 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java
@@ -26,6 +26,7 @@
 import static org.apache.accumulo.core.client.summary.CountingSummarizer.SEEN_STAT;
 import static org.apache.accumulo.core.client.summary.CountingSummarizer.TOO_LONG_STAT;
 import static org.apache.accumulo.core.client.summary.CountingSummarizer.TOO_MANY_STAT;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.test.functional.BasicSummarizer.DELETES_STAT;
 import static org.apache.accumulo.test.functional.BasicSummarizer.MAX_TIMESTAMP_STAT;
 import static org.apache.accumulo.test.functional.BasicSummarizer.MIN_TIMESTAMP_STAT;
@@ -75,6 +76,7 @@
 import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
 import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
 import org.apache.accumulo.core.clientImpl.AccumuloServerException;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -82,8 +84,11 @@
 import org.apache.accumulo.core.iterators.Filter;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.util.FileMetadataUtil;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
@@ -91,9 +96,17 @@
 
 public class SummaryIT extends SharedMiniClusterBase {
 
+  public static class SummaryITConfigCallback implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
+      cfg.setProperty(Property.TSERV_MAXMEM, "80M");
+    }
+  }
+
   @BeforeAll
   public static void setup() throws Exception {
-    SharedMiniClusterBase.startMiniCluster();
+    SharedMiniClusterBase.startMiniClusterWithConfig(new SummaryITConfigCallback());
   }
 
   @AfterAll
@@ -147,7 +160,9 @@
       ntc.enableSummarization(sc1);
       c.tableOperations().create(table, ntc);
 
+      assertEquals(0, FileMetadataUtil.countFiles(getCluster().getServerContext(), table));
       BatchWriter bw = writeData(table, c);
+      assertEquals(0, FileMetadataUtil.countFiles(getCluster().getServerContext(), table));
 
       Collection<Summary> summaries = c.tableOperations().summaries(table).flush(false).retrieve();
       assertEquals(0, summaries.size());
@@ -467,68 +482,14 @@
 
   }
 
-  /**
-   * A compaction strategy that initiates a compaction when {@code foo} occurs more than {@code bar}
-   * in the data. The {@link FooCounter} summary data is used to make the determination.
-   */
-  @SuppressWarnings("removal")
-  public static class FooCS extends org.apache.accumulo.tserver.compaction.CompactionStrategy {
-
-    private boolean compact = false;
-
-    @Override
-    public boolean
-        shouldCompact(org.apache.accumulo.tserver.compaction.MajorCompactionRequest request) {
-      return true;
-    }
-
-    @Override
-    public void
-        gatherInformation(org.apache.accumulo.tserver.compaction.MajorCompactionRequest request) {
-      List<Summary> summaries = request.getSummaries(request.getFiles().keySet(),
-          conf -> conf.getClassName().contains("FooCounter"));
-      if (summaries.size() == 1) {
-        Summary summary = summaries.get(0);
-        Long foos = summary.getStatistics().getOrDefault("foos", 0L);
-        Long bars = summary.getStatistics().getOrDefault("bars", 0L);
-
-        compact = foos > bars;
-      }
-    }
-
-    @Override
-    public org.apache.accumulo.tserver.compaction.CompactionPlan
-        getCompactionPlan(org.apache.accumulo.tserver.compaction.MajorCompactionRequest request) {
-      if (compact) {
-        var cp = new org.apache.accumulo.tserver.compaction.CompactionPlan();
-        cp.inputFiles.addAll(request.getFiles().keySet());
-        return cp;
-      }
-      return null;
-    }
-
-  }
-
   @Test
   public void compactionSelectorTest() throws Exception {
     // Create a compaction config that will filter out foos if there are too many. Uses summary
     // data to know if there are too many foos.
     PluginConfig csc = new PluginConfig(FooSelector.class.getName());
     CompactionConfig compactConfig = new CompactionConfig().setSelector(csc);
-    compactionTest(compactConfig);
-  }
-
-  @SuppressWarnings("removal")
-  @Test
-  public void compactionStrategyTest() throws Exception {
-    var csc =
-        new org.apache.accumulo.core.client.admin.CompactionStrategyConfig(FooCS.class.getName());
-    CompactionConfig compactConfig = new CompactionConfig().setCompactionStrategy(csc);
-    compactionTest(compactConfig);
-  }
-
-  private void compactionTest(CompactionConfig compactConfig) throws Exception {
     final String table = getUniqueNames(1)[0];
+
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       NewTableConfiguration ntc = new NewTableConfiguration();
       SummarizerConfiguration sc1 =
@@ -653,7 +614,7 @@
             assertEquals(1L, (long) summary.getStatistics().getOrDefault("foos", 0L));
             break;
           } catch (AccumuloSecurityException ase) {
-            UtilWaitThread.sleep(500);
+            Thread.sleep(500);
             tries++;
           }
         }
@@ -885,8 +846,8 @@
         // this loop should cause a varying number of files and compactions
         try (BatchWriter bw = c.createBatchWriter(table)) {
           for (int i = 0; i < 10000; i++) {
-            String row = String.format("%06d", random.nextInt(1_000_000));
-            String fam = String.format("%03d", random.nextInt(100));
+            String row = String.format("%06d", RANDOM.get().nextInt(1_000_000));
+            String fam = String.format("%03d", RANDOM.get().nextInt(100));
             String qual = String.format("%06d", q++);
             write(bw, row, fam, qual, "val");
             famCounts.merge(fam, 1L, Long::sum);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
index 22d685b..0f63b9f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TableIT.java
@@ -34,7 +34,7 @@
 import org.apache.accumulo.core.client.admin.TableOperations;
 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.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
@@ -73,7 +73,7 @@
       to.flush(tableName, null, null, true);
       VerifyIngest.verifyIngest(c, params);
       TableId id = TableId.of(to.tableIdMap().get(tableName));
-      try (Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (Scanner s = c.createScanner(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY)) {
         s.setRange(new KeyExtent(id, null, null).toMetaRange());
         s.fetchColumnFamily(DataFileColumnFamily.NAME);
         assertTrue(s.stream().findAny().isPresent());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java
index 86969b3..7b71593 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletMetadataIT.java
@@ -18,13 +18,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.minicluster.ServerType.TABLET_SERVER;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -59,7 +58,7 @@
     try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
       while (c.instanceOperations().getTabletServers().size() != NUM_TSERVERS) {
         log.info("Waiting for tservers to start up...");
-        sleepUninterruptibly(5, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(5));
       }
       Set<TServerInstance> servers = TabletMetadata.getLiveTServers((ClientContext) c);
       assertEquals(NUM_TSERVERS, servers.size());
@@ -70,7 +69,7 @@
 
       while (c.instanceOperations().getTabletServers().size() == NUM_TSERVERS) {
         log.info("Waiting for a tserver to die...");
-        sleepUninterruptibly(5, TimeUnit.SECONDS);
+        Thread.sleep(SECONDS.toMillis(5));
       }
       servers = TabletMetadata.getLiveTServers((ClientContext) c);
       assertEquals(NUM_TSERVERS - 1, servers.size());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index a4c7f06..09290f8 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -18,6 +18,7 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
@@ -52,20 +53,20 @@
 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.ZooUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.server.manager.state.CurrentState;
 import org.apache.accumulo.server.manager.state.MergeInfo;
 import org.apache.accumulo.server.manager.state.MetaDataTableScanner;
 import org.apache.accumulo.server.manager.state.TabletStateChangeIterator;
+import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -105,16 +106,16 @@
       createTable(client, t3, true);
 
       // examine a clone of the metadata table, so we can manipulate it
-      copyTable(client, MetadataTable.NAME, metaCopy1);
+      copyTable(client, AccumuloTable.METADATA.tableName(), metaCopy1);
 
-      State state = new State(client);
-      int tabletsInFlux = findTabletsNeedingAttention(client, metaCopy1, state);
-      while (tabletsInFlux > 0) {
-        log.debug("Waiting for {} tablets for {}", tabletsInFlux, metaCopy1);
-        UtilWaitThread.sleep(500);
-        copyTable(client, MetadataTable.NAME, metaCopy1);
-        tabletsInFlux = findTabletsNeedingAttention(client, metaCopy1, state);
-      }
+      log.debug("Waiting for tablets");
+      final State state = new State(client);
+      Wait.waitFor(() -> findTabletsNeedingAttention(client, metaCopy1, state) <= 0,
+          SECONDS.toMillis(30), 500);
+
+      // update the clone for additional manipulations
+      copyTable(client, AccumuloTable.METADATA.tableName(), metaCopy1);
+
       assertEquals(0, findTabletsNeedingAttention(client, metaCopy1, state),
           "No tables should need attention");
 
@@ -134,7 +135,7 @@
           "Should have one tablet that needs to be unassigned");
 
       // test the cases where there is ongoing merges
-      state = new State(client) {
+      State state2 = new State(client) {
         @Override
         public Collection<MergeInfo> merges() {
           TableId tableIdToModify = TableId.of(client.tableOperations().tableIdMap().get(t3));
@@ -142,13 +143,13 @@
               new MergeInfo(new KeyExtent(tableIdToModify, null, null), MergeInfo.Operation.MERGE));
         }
       };
-      assertEquals(1, findTabletsNeedingAttention(client, metaCopy2, state),
+      assertEquals(1, findTabletsNeedingAttention(client, metaCopy2, state2),
           "Should have 2 tablets that need to be chopped or unassigned");
 
       // test the bad tablet location state case (inconsistent metadata)
-      state = new State(client);
+      state2 = new State(client);
       addDuplicateLocation(client, metaCopy3, t3);
-      assertEquals(1, findTabletsNeedingAttention(client, metaCopy3, state),
+      assertEquals(1, findTabletsNeedingAttention(client, metaCopy3, state2),
           "Should have 1 tablet that needs a metadata repair");
 
       // clean up
@@ -268,9 +269,9 @@
       }
     }
 
-    // metadata should be stable with only 7 rows (1 replication + 2 for each table)
+    // metadata should be stable with only 6 rows (2 for each table)
     log.debug("Gathered {} rows to create copy {}", mutations.size(), copy);
-    assertEquals(7, mutations.size(), "Metadata should have 7 rows (1 repl + 2 for each table)");
+    assertEquals(6, mutations.size(), "Metadata should have 6 rows (2 for each table)");
     client.tableOperations().create(copy);
 
     try (BatchWriter writer = client.createBatchWriter(copy)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TimeoutIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TimeoutIT.java
index fdbfcd8..b1b17a6 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TimeoutIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TimeoutIT.java
@@ -18,12 +18,11 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.time.Duration;
 import java.util.Collections;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -59,10 +58,10 @@
     client.tableOperations().addConstraint(tableName, SlowConstraint.class.getName());
 
     // give constraint time to propagate through zookeeper
-    sleepUninterruptibly(1, TimeUnit.SECONDS);
+    Thread.sleep(SECONDS.toMillis(1));
 
-    BatchWriter bw = client.createBatchWriter(tableName,
-        new BatchWriterConfig().setTimeout(3, TimeUnit.SECONDS));
+    BatchWriter bw =
+        client.createBatchWriter(tableName, new BatchWriterConfig().setTimeout(3, SECONDS));
 
     Mutation mut = new Mutation("r1");
     mut.put("cf1", "cq1", "v1");
@@ -94,7 +93,7 @@
       // should not timeout
       bs.forEach((k, v) -> {});
 
-      bs.setTimeout(5, TimeUnit.SECONDS);
+      bs.setTimeout(5, SECONDS);
       IteratorSetting iterSetting = new IteratorSetting(100, SlowIterator.class);
       iterSetting.addOption("sleepTime", 2000 + "");
       bs.addScanIterator(iterSetting);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.java
deleted file mode 100644
index a528737..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/TooManyDeletesIT.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.test.functional;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.client.summary.Summary;
-import org.apache.accumulo.core.client.summary.summarizers.DeletesSummarizer;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.tserver.compaction.strategies.TooManyDeletesCompactionStrategy;
-import org.junit.jupiter.api.Test;
-
-@SuppressWarnings("removal")
-public class TooManyDeletesIT extends AccumuloClusterHarness {
-  @Test
-  public void tooManyDeletesCompactionStrategyIT() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-
-      String table = getUniqueNames(1)[0];
-
-      SummarizerConfiguration sc = SummarizerConfiguration.builder(DeletesSummarizer.class).build();
-
-      NewTableConfiguration ntc = new NewTableConfiguration().enableSummarization(sc);
-      HashMap<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_COMPACTION_STRATEGY.getKey(),
-          TooManyDeletesCompactionStrategy.class.getName());
-      props.put(Property.TABLE_COMPACTION_STRATEGY_PREFIX.getKey()
-          + TooManyDeletesCompactionStrategy.THRESHOLD_OPT, ".25");
-      // ensure compaction does not happen because of the number of files
-      props.put(Property.TABLE_MAJC_RATIO.getKey(), "10");
-      ntc.setProperties(props);
-
-      c.tableOperations().create(table, ntc);
-
-      try (BatchWriter bw = c.createBatchWriter(table)) {
-        for (int i = 0; i < 1000; i++) {
-          Mutation m = new Mutation("row" + i);
-          m.put("f", "q", "v" + i);
-          bw.addMutation(m);
-        }
-      }
-
-      List<Summary> summaries =
-          c.tableOperations().summaries(table).flush(true).withConfiguration(sc).retrieve();
-      assertEquals(1, summaries.size());
-
-      Summary summary = summaries.get(0);
-
-      assertEquals(1000L, (long) summary.getStatistics().get(DeletesSummarizer.TOTAL_STAT));
-      assertEquals(0L, (long) summary.getStatistics().get(DeletesSummarizer.DELETES_STAT));
-
-      try (BatchWriter bw = c.createBatchWriter(table)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation m = new Mutation("row" + i);
-          m.putDelete("f", "q");
-          bw.addMutation(m);
-        }
-      }
-
-      summaries = c.tableOperations().summaries(table).flush(true).withConfiguration(sc).retrieve();
-      assertEquals(1, summaries.size());
-
-      summary = summaries.get(0);
-
-      assertEquals(1100L, (long) summary.getStatistics().get(DeletesSummarizer.TOTAL_STAT));
-      assertEquals(100L, (long) summary.getStatistics().get(DeletesSummarizer.DELETES_STAT));
-
-      try (BatchWriter bw = c.createBatchWriter(table)) {
-        for (int i = 100; i < 300; i++) {
-          Mutation m = new Mutation("row" + i);
-          m.putDelete("f", "q");
-          bw.addMutation(m);
-        }
-      }
-
-      // after a flush occurs Accumulo will check if a major compaction is needed. This check should
-      // call the compaction strategy, which should decide to compact
-      // all files based on the number of deletes.
-      c.tableOperations().flush(table, null, null, true);
-
-      // wait for the compaction to happen
-      while (true) {
-        // the flush should cause
-        summaries =
-            c.tableOperations().summaries(table).flush(false).withConfiguration(sc).retrieve();
-        assertEquals(1, summaries.size());
-
-        summary = summaries.get(0);
-        long total = summary.getStatistics().get(DeletesSummarizer.TOTAL_STAT);
-        long deletes = summary.getStatistics().get(DeletesSummarizer.DELETES_STAT);
-
-        if (total == 700 && deletes == 0) {
-          // a compaction was triggered based on the number of deletes
-          break;
-        }
-
-        UtilWaitThread.sleep(50);
-      }
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index 22dc863..1f3eea1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -18,13 +18,14 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.conf.Property.GC_CYCLE_DELAY;
 import static org.apache.accumulo.core.conf.Property.GC_CYCLE_START;
 import static org.apache.accumulo.core.conf.Property.INSTANCE_ZK_TIMEOUT;
 import static org.apache.accumulo.core.conf.Property.TSERV_WAL_MAX_SIZE;
 import static org.apache.accumulo.core.conf.Property.TSERV_WAL_REPLICATION;
 import static org.apache.accumulo.core.security.Authorizations.EMPTY;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.apache.accumulo.minicluster.ServerType.GARBAGE_COLLECTOR;
 import static org.apache.accumulo.minicluster.ServerType.TABLET_SERVER;
@@ -39,7 +40,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -49,8 +49,7 @@
 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.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.LogColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
@@ -122,17 +121,18 @@
       assertEquals(3, countInUse(walsAfterRoll.values()), "all WALs should be in use");
 
       // flush the tables
-      for (String table : new String[] {tableName, MetadataTable.NAME, RootTable.NAME}) {
+      for (String table : new String[] {tableName, AccumuloTable.METADATA.tableName(),
+          AccumuloTable.ROOT.tableName()}) {
         c.tableOperations().flush(table, null, null, true);
       }
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
       // rolled WAL is no longer in use, but needs to be GC'd
       Map<String,WalState> walsAfterflush = getWALsAndAssertCount(context, 3);
       assertEquals(2, countInUse(walsAfterflush.values()), "inUse should be 2");
 
       // let the GC run for a little bit
       control.start(GARBAGE_COLLECTOR);
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(5));
       // make sure the unused WAL goes away
       getWALsAndAssertCount(context, 2);
       control.stop(GARBAGE_COLLECTOR);
@@ -145,7 +145,7 @@
       // wait for the metadata table to go back online
       getRecoveryMarkers(c);
       // allow a little time for the manager to notice ASSIGNED_TO_DEAD_SERVER tablets
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(5));
       Map<KeyExtent,List<String>> markers = getRecoveryMarkers(c);
       // log.debug("markers " + markers);
       assertEquals(1, markers.size(), "one tablet should have markers");
@@ -161,7 +161,7 @@
       // log.debug("wals after " + walsAfterRestart);
       assertEquals(4, countInUse(walsAfterRestart.values()), "used WALs after restart should be 4");
       control.start(GARBAGE_COLLECTOR);
-      sleepUninterruptibly(5, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(5));
       Map<String,WalState> walsAfterRestartAndGC = getWALsAndAssertCount(context, 2);
       assertEquals(2, countInUse(walsAfterRestartAndGC.values()), "logs in use should be 2");
     }
@@ -182,11 +182,11 @@
       byte[] value = new byte[10];
 
       for (int r = 0; r < row; r++) {
-        random.nextBytes(rowData);
+        RANDOM.get().nextBytes(rowData);
         Mutation m = new Mutation(rowData);
         for (int c = 0; c < col; c++) {
-          random.nextBytes(cq);
-          random.nextBytes(value);
+          RANDOM.get().nextBytes(cq);
+          RANDOM.get().nextBytes(value);
           m.put(CF, new Text(cq), new Value(value));
         }
         bw.addMutation(m);
@@ -199,8 +199,8 @@
 
   private Map<KeyExtent,List<String>> getRecoveryMarkers(AccumuloClient c) throws Exception {
     Map<KeyExtent,List<String>> result = new HashMap<>();
-    try (Scanner root = c.createScanner(RootTable.NAME, EMPTY);
-        Scanner meta = c.createScanner(MetadataTable.NAME, EMPTY)) {
+    try (Scanner root = c.createScanner(AccumuloTable.ROOT.tableName(), EMPTY);
+        Scanner meta = c.createScanner(AccumuloTable.METADATA.tableName(), EMPTY)) {
       root.setRange(TabletsSection.getRange());
       root.fetchColumnFamily(LogColumnFamily.NAME);
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(root);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
index 85414a4..19d6c8f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
@@ -27,13 +27,14 @@
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 // ACCUMULO-2757 - make sure we don't make too many more watchers
@@ -62,9 +63,9 @@
       }
       c.tableOperations().list();
       String zooKeepers = ClientProperty.INSTANCE_ZOOKEEPERS.getValue(props);
-      // base number of watchers 110 to 125, and 15 to 20 per table in a single-node instance.
-      final long MIN = 150L;
-      final long MAX = 250L;
+      // base number of watchers 90 to 125, and 15 to 25 per table in a single-node instance.
+      final long MIN = 135L;
+      final long MAX = 200L;
       long total = 0;
       final HostAndPort hostAndPort = HostAndPort.fromString(zooKeepers);
       for (int i = 0; i < 5; i++) {
@@ -78,11 +79,13 @@
           if (total > MIN && total < MAX) {
             break;
           }
-          log.debug("Expected number of watchers to be contained in ({}, {}), but"
-              + " actually was {}. Sleeping and retrying", MIN, MAX, total);
+          log.debug("Expected number of watchers to be contained in ({}, {}), currently have {}"
+              + ". Sleeping and retrying", MIN, MAX, total);
           Thread.sleep(5000);
         }
       }
+      log.info("Number of watchers to be expected in range: ({}, {}), currently have {}", MIN, MAX,
+          total);
 
       assertTrue(total > MIN && total < MAX, "Expected number of watchers to be contained in ("
           + MIN + ", " + MAX + "), but actually was " + total);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 30d8331..dce3d5d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -18,34 +18,31 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.apache.accumulo.harness.AccumuloITBase.random;
+import static java.util.concurrent.TimeUnit.DAYS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.util.HashMap;
 import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockWatcher;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.LockWatcher;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 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.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.client.ClientServiceHandler;
@@ -58,6 +55,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
@@ -68,7 +67,7 @@
 
   public static class ZombieTServerThriftClientHandler
       extends org.apache.accumulo.test.performance.NullTserver.NullTServerTabletClientHandler
-      implements TabletClientService.Iface, TabletScanClientService.Iface {
+      implements TabletServerClientService.Iface, TabletScanClientService.Iface {
 
     int statusCount = 0;
 
@@ -89,7 +88,12 @@
           return result;
         }
       }
-      sleepUninterruptibly(Integer.MAX_VALUE, TimeUnit.DAYS);
+      try {
+        Thread.sleep(DAYS.toMillis(Integer.MAX_VALUE));
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        log.info("probably received shutdown, interrupted during infinite sleep", ex);
+      }
       return null;
     }
 
@@ -104,7 +108,7 @@
   private static final Logger log = LoggerFactory.getLogger(ZombieTServer.class);
 
   public static void main(String[] args) throws Exception {
-    int port = random.nextInt(30000) + 2000;
+    int port = RANDOM.get().nextInt(30000) + 2000;
     var context = new ServerContext(SiteConfiguration.auto());
     final ClientServiceHandler csh =
         new ClientServiceHandler(context, new TransactionWatcher(context));
@@ -115,12 +119,11 @@
         ThriftProcessorTypes.CLIENT.getTProcessor(ClientService.Processor.class,
             ClientService.Iface.class, csh, context));
     muxProcessor.registerProcessor(ThriftClientTypes.TABLET_SERVER.getServiceName(),
-        ThriftProcessorTypes.TABLET_SERVER.getTProcessor(TabletClientService.Processor.class,
-            TabletClientService.Iface.class, tch, context));
-    muxProcessor.registerProcessor(ThriftProcessorTypes.TABLET_SERVER_SCAN.getServiceName(),
-        ThriftProcessorTypes.TABLET_SERVER_SCAN.getTProcessor(
-            TabletScanClientService.Processor.class, TabletScanClientService.Iface.class, tch,
-            context));
+        ThriftProcessorTypes.TABLET_SERVER.getTProcessor(TabletServerClientService.Processor.class,
+            TabletServerClientService.Iface.class, tch, context));
+    muxProcessor.registerProcessor(ThriftProcessorTypes.TABLET_SCAN.getServiceName(),
+        ThriftProcessorTypes.TABLET_SCAN.getTProcessor(TabletScanClientService.Processor.class,
+            TabletScanClientService.Iface.class, tch, context));
 
     ServerAddress serverPort = TServerUtils.startTServer(context.getConfiguration(),
         ThriftServerType.CUSTOM_HS_HA, muxProcessor, "ZombieTServer", "walking dead", 2,
@@ -129,6 +132,7 @@
         HostAndPort.fromParts("0.0.0.0", port));
 
     String addressString = serverPort.address.toString();
+
     var zLockPath =
         ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + addressString);
     ZooReaderWriter zoo = context.getZooReaderWriter();
@@ -167,9 +171,8 @@
       }
     };
 
-    byte[] lockContent =
-        new ServerServices(addressString, Service.TSERV_CLIENT).toString().getBytes(UTF_8);
-    if (zlock.tryLock(lw, lockContent)) {
+    if (zlock.tryLock(lw, new ServiceLockData(UUID.randomUUID(), addressString, ThriftService.TSERV,
+        ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME))) {
       log.debug("Obtained tablet server lock {}", zlock.getLockPath());
     }
     // modify metadata
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
index 1e84df7..f918070 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
@@ -18,13 +18,12 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.time.Duration;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -71,7 +70,7 @@
       }
 
       // give the servers time to react
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
 
       // start zookeeper back up
       cluster.start();
diff --git a/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java b/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
deleted file mode 100644
index 580c690..0000000
--- a/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
+++ /dev/null
@@ -1,189 +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.test.gc.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.data.InstanceId;
-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.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.gc.replication.CloseWriteAheadLogReferences;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class CloseWriteAheadLogReferencesIT extends ConfigurableMacBase {
-
-  private WrappedCloseWriteAheadLogReferences refs;
-  private AccumuloClient client;
-
-  private static class WrappedCloseWriteAheadLogReferences extends CloseWriteAheadLogReferences {
-    public WrappedCloseWriteAheadLogReferences(ServerContext context) {
-      super(context);
-    }
-
-    @Override
-    protected long updateReplicationEntries(AccumuloClient client, Set<String> closedWals) {
-      return super.updateReplicationEntries(client, closedWals);
-    }
-  }
-
-  @BeforeEach
-  public void setupInstance() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    client.securityOperations().grantTablePermission(client.whoami(), MetadataTable.NAME,
-        TablePermission.WRITE);
-    ReplicationTable.setOnline(client);
-  }
-
-  @AfterEach
-  public void teardownInstance() {
-    client.close();
-  }
-
-  @BeforeEach
-  public void setupEasyMockStuff() {
-    SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
-    final AccumuloConfiguration systemConf = new ConfigurationCopy(new HashMap<>());
-
-    // Just make the SiteConfiguration delegate to our AccumuloConfiguration
-    // Presently, we only need get(Property) and iterator().
-    EasyMock.expect(siteConfig.get(EasyMock.anyObject(Property.class))).andAnswer(() -> {
-      Object[] args = EasyMock.getCurrentArguments();
-      return systemConf.get((Property) args[0]);
-    }).anyTimes();
-    EasyMock.expect(siteConfig.getBoolean(EasyMock.anyObject(Property.class))).andAnswer(() -> {
-      Object[] args = EasyMock.getCurrentArguments();
-      return systemConf.getBoolean((Property) args[0]);
-    }).anyTimes();
-
-    EasyMock.expect(siteConfig.iterator()).andAnswer(systemConf::iterator).anyTimes();
-    ServerContext context = createMock(ServerContext.class);
-    expect(context.getProperties()).andReturn(new Properties()).anyTimes();
-    expect(context.getZooKeepers()).andReturn("localhost").anyTimes();
-    expect(context.getInstanceName()).andReturn("test").anyTimes();
-    expect(context.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
-    expect(context.getInstanceID()).andReturn(InstanceId.of("1111")).anyTimes();
-    expect(context.getZooKeeperRoot()).andReturn(Constants.ZROOT + "/1111").anyTimes();
-
-    replay(siteConfig, context);
-
-    refs = new WrappedCloseWriteAheadLogReferences(context);
-  }
-
-  @Test
-  public void unclosedWalsLeaveStatusOpen() throws Exception {
-    Set<String> wals = Collections.emptySet();
-    try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
-      Mutation m =
-          new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wal/tserver+port/12345");
-      m.put(ReplicationSection.COLF, new Text("1"),
-          StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-      bw.addMutation(m);
-    }
-
-    refs.updateReplicationEntries(client, wals);
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      Entry<Key,Value> entry = getOnlyElement(s);
-      Status status = Status.parseFrom(entry.getValue().get());
-      assertFalse(status.getClosed());
-    }
-  }
-
-  @Test
-  public void closedWalsUpdateStatus() throws Exception {
-    String file = "file:/accumulo/wal/tserver+port/12345";
-    Set<String> wals = Collections.singleton(file);
-    try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text("1"),
-          StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-      bw.addMutation(m);
-    }
-
-    refs.updateReplicationEntries(client, wals);
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME)) {
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      Entry<Key,Value> entry = getOnlyElement(s);
-      Status status = Status.parseFrom(entry.getValue().get());
-      assertTrue(status.getClosed());
-    }
-  }
-
-  @Test
-  public void partiallyReplicatedReferencedWalsAreNotClosed() throws Exception {
-    String file = "file:/accumulo/wal/tserver+port/12345";
-    Set<String> wals = Collections.singleton(file);
-    try (BatchWriter bw = ReplicationTable.getBatchWriter(client)) {
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, TableId.of("1"), ProtobufUtil.toValue(StatusUtil.ingestedUntil(1000)));
-      bw.addMutation(m);
-    }
-
-    refs.updateReplicationEntries(client, wals);
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      Entry<Key,Value> entry = getOnlyElement(s);
-      Status status = Status.parseFrom(entry.getValue().get());
-      assertFalse(status.getClosed());
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java b/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java
similarity index 86%
rename from test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java
rename to test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java
index 7069ed5..3255ae7 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/lock/ServiceLockIT.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.test.fate.zookeeper;
+package org.apache.accumulo.test.lock;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.junit.jupiter.api.Assertions.assertAll;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -31,19 +32,23 @@
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.LockSupport;
 
 import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.AccumuloLockWatcher;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.LockLossReason;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooSession;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock.AccumuloLockWatcher;
+import org.apache.accumulo.core.lock.ServiceLock.LockLossReason;
+import org.apache.accumulo.core.lock.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.test.util.Wait;
 import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
 import org.apache.zookeeper.CreateMode;
@@ -229,7 +234,8 @@
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -253,7 +259,8 @@
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -278,7 +285,8 @@
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -311,7 +319,8 @@
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -324,7 +333,8 @@
 
     TestALW lw2 = new TestALW();
 
-    zl2.lock(lw2, "test2".getBytes(UTF_8));
+    zl2.lock(lw2, new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     assertFalse(lw2.locked);
     assertFalse(zl2.isLocked());
@@ -333,7 +343,8 @@
 
     TestALW lw3 = new TestALW();
 
-    zl3.lock(lw3, "test3".getBytes(UTF_8));
+    zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), "test3", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     List<String> children = ServiceLock.validateAndSort(parent, zk.getChildren(parent.toString()));
 
@@ -386,7 +397,8 @@
 
       TestALW lw = new TestALW();
 
-      zl.lock(lw, "test1".getBytes(UTF_8));
+      zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME));
 
       lw.waitForChanges(1);
 
@@ -431,7 +443,8 @@
       final RetryLockWatcher zlw1 = new RetryLockWatcher();
       ServiceLock zl1 =
           getZooLock(zk1, parent, UUID.fromString("00000000-0000-0000-0000-aaaaaaaaaaaa"));
-      zl1.lock(zlw1, "test1".getBytes(UTF_8));
+      zl1.lock(zlw1, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME));
       // The call above creates two nodes in ZK because of the overridden create method in
       // ZooKeeperWrapper.
       // The nodes created are:
@@ -446,7 +459,8 @@
       final RetryLockWatcher zlw2 = new RetryLockWatcher();
       ServiceLock zl2 =
           getZooLock(zk2, parent, UUID.fromString("00000000-0000-0000-0000-bbbbbbbbbbbb"));
-      zl2.lock(zlw2, "test1".getBytes(UTF_8));
+      zl2.lock(zlw2, new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME));
       // The call above creates two nodes in ZK because of the overridden create method in
       // ZooKeeperWrapper.
       // The nodes created are:
@@ -530,7 +544,8 @@
           ServiceLock zl = getZooLock(zk, parent, uuid);
           getLockLatch.countDown(); // signal we are done
           getLockLatch.await(); // wait for others to finish
-          zl.lock(lockWatcher, "test1".getBytes(UTF_8)); // race to the lock
+          zl.lock(lockWatcher, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+              ServiceDescriptor.DEFAULT_GROUP_NAME)); // race to the lock
           lockCompletedLatch.countDown();
           unlockLatch.await();
           zl.unlock();
@@ -575,8 +590,9 @@
     try (ZooKeeperWrapper zk = new ZooKeeperWrapper(szk.getConn(), 30000, watcher)) {
       ZooUtil.digestAuth(zk, "secret");
 
-      Wait.waitFor(() -> !watcher.isConnected(), 30_000, 50);
-
+      while (!watcher.isConnected()) {
+        Thread.sleep(50);
+      }
       // Create the parent node
       zk.createOnce(parent.toString(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
           CreateMode.PERSISTENT);
@@ -613,12 +629,13 @@
         String first = children.get(0);
         int workerWithLock = parseLockWorkerName(first);
         LockWorker worker = workers.get(workerWithLock);
-        assertTrue(worker.holdsLock());
-        workers.forEach(w -> {
-          if (w != worker) {
-            assertFalse(w.holdsLock());
-          }
-        });
+        assertAll(() -> assertTrue(worker.holdsLock(),
+            "Expected worker, " + worker + " did not hold lock"), () -> workers.forEach(w -> {
+              if (w != worker) {
+                assertFalse(w.holdsLock(),
+                    "Expected worker, " + worker + " to hold lock. Instead " + w + " holds lock");
+              }
+            }));
         worker.unlock();
         Thread.sleep(100); // need to wait here so that the watchers fire.
       }
@@ -656,7 +673,8 @@
 
       TestALW lw = new TestALW();
 
-      boolean ret = zl.tryLock(lw, "test1".getBytes(UTF_8));
+      boolean ret = zl.tryLock(lw, new ServiceLockData(UUID.randomUUID(), "test1",
+          ThriftService.TSERV, ServiceDescriptor.DEFAULT_GROUP_NAME));
 
       assertTrue(ret);
 
@@ -688,11 +706,18 @@
 
       TestALW lw = new TestALW();
 
-      zl.lock(lw, "test1".getBytes(UTF_8));
-      assertEquals("test1", new String(zk.getData(zl.getLockPath(), null, null)));
+      ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME);
+      zl.lock(lw, sld1);
+      assertEquals(Optional.of(sld1),
+          ServiceLockData.parse(zk.getData(zl.getLockPath(), null, null)));
 
-      zl.replaceLockData("test2".getBytes(UTF_8));
-      assertEquals("test2", new String(zk.getData(zl.getLockPath(), null, null)));
+      ServiceLockData sld2 = new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME);
+      zl.replaceLockData(sld2);
+      assertEquals(Optional.of(sld2),
+          ServiceLockData.parse(zk.getData(zl.getLockPath(), null, null)));
     }
   }
+
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
index 21173c3..3b0e648 100644
--- a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
@@ -23,6 +23,7 @@
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -36,18 +37,16 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.manager.thrift.ManagerState;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.manager.state.MergeStats;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.manager.state.Assignment;
@@ -59,6 +58,8 @@
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
+import com.google.common.net.HostAndPort;
+
 public class MergeStateIT extends ConfigurableMacBase {
 
   private static class MockCurrentState implements CurrentState {
@@ -104,7 +105,7 @@
 
   private static void update(AccumuloClient c, Mutation m)
       throws TableNotFoundException, MutationsRejectedException {
-    try (BatchWriter bw = c.createBatchWriter(MetadataTable.NAME)) {
+    try (BatchWriter bw = c.createBatchWriter(AccumuloTable.METADATA.tableName())) {
       bw.addMutation(m);
     }
   }
@@ -114,8 +115,8 @@
     ServerContext context = getServerContext();
     try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProperties()).build()) {
       accumuloClient.securityOperations().grantTablePermission(accumuloClient.whoami(),
-          MetadataTable.NAME, TablePermission.WRITE);
-      BatchWriter bw = accumuloClient.createBatchWriter(MetadataTable.NAME);
+          AccumuloTable.METADATA.tableName(), TablePermission.WRITE);
+      BatchWriter bw = accumuloClient.createBatchWriter(AccumuloTable.METADATA.tableName());
 
       // Create a fake METADATA table with these splits
       String[] splits = {"a", "e", "j", "o", "t", "z"};
@@ -128,7 +129,6 @@
             TabletColumnFamily.createPrevRowMutation(new KeyExtent(tableId, split, pr));
         prevRow.put(CurrentLocationColumnFamily.NAME, new Text("123456"),
             new Value("127.0.0.1:1234"));
-        ChoppedColumnFamily.CHOPPED_COLUMN.put(prevRow, new Value("junk"));
         bw.addMutation(prevRow);
         pr = split;
       }
@@ -171,8 +171,8 @@
       assertEquals(MergeState.WAITING_FOR_OFFLINE, newState);
 
       // unassign the tablets
-      try (BatchDeleter deleter =
-          accumuloClient.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 1000)) {
+      try (BatchDeleter deleter = accumuloClient
+          .createBatchDeleter(AccumuloTable.METADATA.tableName(), Authorizations.EMPTY, 1000)) {
         deleter.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
         deleter.setRanges(Collections.singletonList(new Range()));
         deleter.delete();
@@ -190,30 +190,20 @@
       metaDataStateStore
           .setLocations(Collections.singletonList(new Assignment(tablet, state.someTServer, null)));
 
-      // onos... there's a new tablet online
-      stats = scan(state, metaDataStateStore);
-      assertEquals(MergeState.WAITING_FOR_CHOPPED, stats.nextMergeState(accumuloClient, state));
-
-      // chop it
-      m = TabletColumnFamily.createPrevRowMutation(tablet);
-      ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("junk"));
-      update(accumuloClient, m);
-
       stats = scan(state, metaDataStateStore);
       assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(accumuloClient, state));
 
       // take it offline
       m = TabletColumnFamily.createPrevRowMutation(tablet);
-      Collection<Collection<String>> walogs = Collections.emptyList();
+      Collection<LogEntry> walogs = Collections.emptyList();
       metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null,
-          Location.current(state.someTServer), null, null, walogs, false)), null);
+          Location.current(state.someTServer), null, null, walogs)), null);
 
       // Add a walog which should keep the state from transitioning to MERGING
       KeyExtent ke = new KeyExtent(tableId, new Text("t"), new Text("p"));
       m = new Mutation(ke.toMetaRow());
-      LogEntry logEntry = new LogEntry(ke, 100, "f1");
-      m.at().family(logEntry.getColumnFamily()).qualifier(logEntry.getColumnQualifier())
-          .timestamp(logEntry.timestamp).put(logEntry.getValue());
+      LogEntry logEntry = LogEntry.fromPath("localhost+1234/" + UUID.randomUUID());
+      logEntry.addToMutation(m);
       update(accumuloClient, m);
 
       // Verify state is still WAITING_FOR_OFFLINE
@@ -223,7 +213,7 @@
 
       // Delete the walog which will now allow a transition to MERGING
       m = new Mutation(ke.toMetaRow());
-      m.putDelete(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.timestamp);
+      logEntry.deleteFromMutation(m);
       update(accumuloClient, m);
 
       // now we can split
@@ -236,7 +226,7 @@
     MergeStats stats = new MergeStats(state.mergeInfo);
     stats.getMergeInfo().setState(MergeState.WAITING_FOR_OFFLINE);
     for (TabletLocationState tss : metaDataStateStore) {
-      stats.update(tss.extent, tss.getState(state.onlineTabletServers()), tss.chopped, false);
+      stats.update(tss.extent, tss.getState(state.onlineTabletServers()));
     }
     return stats;
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
index 764bc1d..8ebb753 100644
--- a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.test.manager;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
@@ -53,14 +54,12 @@
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
-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.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.spi.balancer.HostRegexTableLoadBalancer;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.miniclusterImpl.ProcessReference;
@@ -77,6 +76,7 @@
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.SetMultimap;
+import com.google.common.net.HostAndPort;
 
 public class SuspendedTabletsIT extends ConfigurableMacBase {
   private static final Logger log = LoggerFactory.getLogger(SuspendedTabletsIT.class);
@@ -121,16 +121,19 @@
       }
       HostAndPort metadataServer = HostAndPort.fromString(tservers.get(0));
       log.info("Configuring balancer to assign all metadata tablets to {}", metadataServer);
-      iops.setProperty(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + MetadataTable.NAME,
+      iops.setProperty(
+          HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + AccumuloTable.METADATA.tableName(),
           metadataServer.toString());
 
       // Wait for the balancer to assign all metadata tablets to the chosen server.
       ClientContext ctx = (ClientContext) client;
-      TabletLocations tl = TabletLocations.retrieve(ctx, MetadataTable.NAME, RootTable.NAME);
+      TabletLocations tl = TabletLocations.retrieve(ctx, AccumuloTable.METADATA.tableName(),
+          AccumuloTable.ROOT.tableName());
       while (tl.hosted.keySet().size() != 1 || !tl.hosted.containsKey(metadataServer)) {
         log.info("Metadata tablets are not hosted on the correct server. Waiting for balancer...");
         Thread.sleep(1000L);
-        tl = TabletLocations.retrieve(ctx, MetadataTable.NAME, RootTable.NAME);
+        tl = TabletLocations.retrieve(ctx, AccumuloTable.METADATA.tableName(),
+            AccumuloTable.ROOT.tableName());
       }
       log.info("Metadata tablets are now hosted on {}", metadataServer);
     }
@@ -305,11 +308,10 @@
     @Override
     public void eliminateTabletServers(ClientContext ctx, TabletLocations locs, int count)
         throws Exception {
-
       Set<TServerInstance> tserverSet = new HashSet<>();
       Set<TServerInstance> metadataServerSet = new HashSet<>();
 
-      TabletLocator tl = TabletLocator.getLocator(ctx, MetadataTable.ID);
+      TabletLocator tl = TabletLocator.getLocator(ctx, AccumuloTable.METADATA.tableId());
       for (TabletLocationState tls : locs.locationStates.values()) {
         if (tls.current != null) {
           // add to set of all servers
@@ -319,8 +321,8 @@
           TabletLocator.TabletLocation tab =
               tl.locateTablet(ctx, tls.extent.toMetaRow(), false, false);
           // add it to the set of servers with metadata
-          metadataServerSet
-              .add(new TServerInstance(tab.tablet_location, Long.valueOf(tab.tablet_session, 16)));
+          metadataServerSet.add(new TServerInstance(tab.getTserverLocation(),
+              Long.valueOf(tab.getTserverSession(), 16)));
         }
       }
 
@@ -332,7 +334,7 @@
           "Expecting " + (TSERVERS - 1) + " tServers in shutdown-list");
 
       List<TServerInstance> tserversList = new ArrayList<>(tserverSet);
-      Collections.shuffle(tserversList, random);
+      Collections.shuffle(tserversList, RANDOM.get());
 
       for (int i1 = 0; i1 < count; ++i1) {
         final String tserverName = tserversList.get(i1).getHostPortSession();
@@ -363,7 +365,6 @@
         }
       }
       throw new IllegalStateException("Tablet servers didn't die!");
-
     }
   }
 
@@ -376,7 +377,7 @@
       // kill tablet servers that are not hosting the metadata table.
       List<ProcessReference> procs = getCluster().getProcesses().get(ServerType.TABLET_SERVER)
           .stream().filter(p -> !metadataTserverProcess.equals(p)).collect(Collectors.toList());
-      Collections.shuffle(procs, random);
+      Collections.shuffle(procs, RANDOM.get());
       assertEquals(TSERVERS - 1, procs.size(), "Not enough tservers exist");
       assertTrue(procs.size() >= count, "Attempting to kill more tservers (" + count
           + ") than exist in the cluster (" + procs.size() + ")");
@@ -454,7 +455,7 @@
 
     public static TabletLocations retrieve(final ClientContext ctx, final String tableName)
         throws Exception {
-      return retrieve(ctx, tableName, MetadataTable.NAME);
+      return retrieve(ctx, tableName, AccumuloTable.METADATA.tableName());
     }
 
     public static TabletLocations retrieve(final ClientContext ctx, final String tableName,
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
deleted file mode 100644
index 8d48ba7..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
+++ /dev/null
@@ -1,234 +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.test.mapred;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.io.File;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.sample.RowSampler;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-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.file.FileSKVIterator;
-import org.apache.accumulo.core.file.rfile.RFileOperations;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.IdentityMapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
-  private static final Logger log = LoggerFactory.getLogger(AccumuloFileOutputFormatIT.class);
-  private static final int JOB_VISIBILITY_CACHE_SIZE = 3000;
-  private static final String PREFIX = AccumuloFileOutputFormatIT.class.getSimpleName();
-  private static final String BAD_TABLE = PREFIX + "_mapred_bad_table";
-  private static final String TEST_TABLE = PREFIX + "_mapred_test_table";
-  private static final String EMPTY_TABLE = PREFIX + "_mapred_empty_table";
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static final SamplerConfiguration SAMPLER_CONFIG =
-      new SamplerConfiguration(RowSampler.class.getName()).addOption("hasher", "murmur3_32")
-          .addOption("modulus", "3");
-
-  @TempDir
-  private static File tempDir;
-
-  @Test
-  public void testEmptyWrite() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(EMPTY_TABLE);
-      handleWriteTests(false);
-    }
-  }
-
-  @Test
-  public void testRealWrite() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(TEST_TABLE);
-      try (BatchWriter bw = c.createBatchWriter(TEST_TABLE)) {
-        Mutation m = new Mutation("Key");
-        m.put("", "", "");
-        bw.addMutation(m);
-      }
-      handleWriteTests(true);
-    }
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    private static class BadKeyMapper implements Mapper<Key,Value,Key,Value> {
-      int index = 0;
-
-      @Override
-      public void map(Key key, Value value, OutputCollector<Key,Value> output, Reporter reporter) {
-        try {
-          try {
-            output.collect(key, value);
-            if (index == 2) {
-              fail();
-            }
-          } catch (Exception e) {
-            log.error(e.toString(), e);
-            assertEquals(2, index);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        index++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() {
-        try {
-          assertEquals(2, index);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 2) {
-        throw new IllegalArgumentException(
-            "Usage : " + MRTester.class.getName() + " <table> <outputfile>");
-      }
-
-      String table = args[0];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-      org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator
-          .setVisibilityCacheSize(job, JOB_VISIBILITY_CACHE_SIZE);
-
-      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat.setOutputPath(job,
-          new Path(args[1]));
-      org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat.setSampler(job,
-          SAMPLER_CONFIG);
-
-      job.setMapperClass(BAD_TABLE.equals(table) ? BadKeyMapper.class : IdentityMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  private void handleWriteTests(boolean content) throws Exception {
-    File f = new File(tempDir, testName());
-    if (f.delete()) {
-      log.debug("Deleted {}", f);
-    }
-    MRTester.main(new String[] {content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
-
-    assertTrue(f.exists());
-    File[] files = f.listFiles(file -> file.getName().startsWith("part-m-"));
-    assertNotNull(files);
-    if (content) {
-      assertEquals(1, files.length);
-      assertTrue(files[0].exists());
-
-      Configuration conf = cluster.getServerContext().getHadoopConf();
-      DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
-      FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
-              NoCryptoServiceFactory.NONE)
-          .withTableConfiguration(acuconf).build()
-          .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
-      assertNotNull(sample);
-    } else {
-      assertEquals(0, files.length);
-    }
-  }
-
-  @Test
-  public void writeBadVisibility() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(BAD_TABLE);
-      try (BatchWriter bw = c.createBatchWriter(BAD_TABLE)) {
-        Mutation m = new Mutation("r1");
-        m.put("cf1", "cq1", "A&B");
-        m.put("cf1", "cq1", "A&B");
-        m.put("cf1", "cq2", "A&");
-        bw.addMutation(m);
-      }
-      File f = new File(tempDir, testName());
-      if (f.delete()) {
-        log.debug("Deleted {}", f);
-      }
-      MRTester.main(new String[] {BAD_TABLE, f.getAbsolutePath()});
-      assertNull(e1);
-      assertNull(e2);
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
deleted file mode 100644
index 0a7876f..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
+++ /dev/null
@@ -1,265 +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.test.mapred;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.io.File;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.sample.RowSampler;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-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.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Level;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormatIT extends AccumuloClusterHarness {
-
-  @BeforeAll
-  public static void setupClass() {
-    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
-  }
-
-  private static AssertionError e1 = null;
-  private static int e1Count = 0;
-  private static AssertionError e2 = null;
-  private static int e2Count = 0;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) {
-        try {
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-          e1Count++;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-          e2Count++;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 1 && args.length != 3) {
-        throw new IllegalArgumentException(
-            "Usage : " + MRTester.class.getName() + " <table> [<batchScan> <scan sample>]");
-      }
-
-      String table = args[0];
-      boolean batchScan = false;
-      boolean sample = false;
-      if (args.length == 3) {
-        batchScan = Boolean.parseBoolean(args[1]);
-        sample = Boolean.parseBoolean(args[2]);
-      }
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setBatchScan(job, batchScan);
-      if (sample) {
-        org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setSamplerConfiguration(job,
-            SAMPLER_CONFIG);
-      }
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String... args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    String table = getUniqueNames(1)[0];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table);
-      try (BatchWriter bw = c.createBatchWriter(table)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-          m.put("", "", String.format("%09x", i));
-          bw.addMutation(m);
-        }
-      }
-
-      e1 = null;
-      e2 = null;
-
-      MRTester.main(table);
-      assertNull(e1);
-      assertNull(e2);
-    }
-  }
-
-  private static final SamplerConfiguration SAMPLER_CONFIG =
-      new SamplerConfiguration(RowSampler.class.getName()).addOption("hasher", "murmur3_32")
-          .addOption("modulus", "3");
-
-  @Test
-  public void testSample() throws Exception {
-    final String TEST_TABLE_3 = getUniqueNames(1)[0];
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(TEST_TABLE_3,
-          new NewTableConfiguration().enableSampling(SAMPLER_CONFIG));
-      try (BatchWriter bw = c.createBatchWriter(TEST_TABLE_3)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-          m.put("", "", String.format("%09x", i));
-          bw.addMutation(m);
-        }
-      }
-
-      MRTester.main(TEST_TABLE_3, "False", "True");
-      assertEquals(38, e1Count);
-      assertEquals(1, e2Count);
-
-      e2Count = e1Count = 0;
-      MRTester.main(TEST_TABLE_3, "False", "False");
-      assertEquals(0, e1Count);
-      assertEquals(0, e2Count);
-
-      e2Count = e1Count = 0;
-      MRTester.main(TEST_TABLE_3, "True", "True");
-      assertEquals(38, e1Count);
-      assertEquals(1, e2Count);
-    }
-  }
-
-  @Test
-  public void testCorrectRangeInputSplits() throws Exception {
-    JobConf job = new JobConf();
-
-    String table = getUniqueNames(1)[0];
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns =
-        Collections.singleton(new Pair<>(new Text("foo"), new Text("bar")));
-    boolean isolated = true, localIters = true;
-    Level level = Level.WARN;
-
-    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
-      accumuloClient.tableOperations().create(table);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setScanAuthorizations(job, auths);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setScanIsolation(job, isolated);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setLocalIterators(job, localIters);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.fetchColumns(job, fetchColumns);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setLogLevel(job, level);
-
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat aif =
-          new org.apache.accumulo.core.client.mapred.AccumuloInputFormat();
-
-      InputSplit[] splits = aif.getSplits(job, 1);
-
-      assertEquals(1, splits.length);
-
-      InputSplit split = splits[0];
-
-      assertEquals(org.apache.accumulo.core.client.mapred.RangeInputSplit.class, split.getClass());
-
-      org.apache.accumulo.core.client.mapred.RangeInputSplit risplit =
-          (org.apache.accumulo.core.client.mapred.RangeInputSplit) split;
-
-      assertEquals(table, risplit.getTableName());
-      assertEquals(isolated, risplit.isIsolatedScan());
-      assertEquals(localIters, risplit.usesLocalIterators());
-      assertEquals(fetchColumns, risplit.getFetchedColumns());
-      assertEquals(level, risplit.getLogLevel());
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
deleted file mode 100644
index 1d0c3d7..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.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.test.mapred;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-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.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormatIT extends AccumuloClusterHarness {
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) {
-        try {
-          String tableName =
-              ((org.apache.accumulo.core.client.mapred.RangeInputSplit) reporter.getInputSplit())
-                  .getTableName();
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
-          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 2) {
-        throw new IllegalArgumentException(
-            "Usage : " + MRTester.class.getName() + " <table1> <table2>");
-      }
-
-      String table1 = args[0];
-      String table2 = args[1];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(
-          org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig1 =
-          new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig2 =
-          new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
-
-      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configMap =
-          new HashMap<>();
-      configMap.put(table1, tableConfig1);
-      configMap.put(table2, tableConfig2);
-
-      org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.setInputTableConfigs(job,
-          configMap);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    String[] tableNames = getUniqueNames(2);
-    String table1 = tableNames[0];
-    String table2 = tableNames[1];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table1);
-      c.tableOperations().create(table2);
-      try (BatchWriter bw = c.createBatchWriter(table1);
-          BatchWriter bw2 = c.createBatchWriter(table2)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation t1m = new Mutation(new Text(String.format("%s_%09x", table1, i + 1)));
-          t1m.put("", "", String.format("%s_%09x", table1, i));
-          bw.addMutation(t1m);
-          Mutation t2m = new Mutation(new Text(String.format("%s_%09x", table2, i + 1)));
-          t2m.put("", "", String.format("%s_%09x", table2, i));
-          bw2.addMutation(t2m);
-        }
-      }
-
-      MRTester.main(new String[] {table1, table2});
-      assertNull(e1);
-      assertNull(e2);
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
deleted file mode 100644
index 8d355ee..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
+++ /dev/null
@@ -1,243 +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.test.mapred;
-
-import static com.google.common.collect.MoreCollectors.onlyElement;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.conf.Property;
-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.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormatIT extends ConfigurableMacBase {
-
-  @Override
-  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "1");
-    cfg.setNumTservers(1);
-  }
-
-  // Prevent regression of ACCUMULO-3709.
-  @Test
-  public void testMapred() throws Exception {
-    Properties props = getClientProperties();
-    try (AccumuloClient client = Accumulo.newClient().from(props).build()) {
-      // create a table and put some data in it
-      client.tableOperations().create(testName());
-
-      JobConf job = new JobConf();
-      BatchWriterConfig batchConfig = new BatchWriterConfig();
-      // no flushes!!!!!
-      batchConfig.setMaxLatency(0, TimeUnit.MILLISECONDS);
-      // use a single thread to ensure our update session times out
-      batchConfig.setMaxWriteThreads(1);
-      // set the max memory so that we ensure we don't flush on the write.
-      batchConfig.setMaxMemory(Long.MAX_VALUE);
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat outputFormat =
-          new org.apache.accumulo.core.client.mapred.AccumuloOutputFormat();
-      ClientInfo ci = ClientInfo.from(props);
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setBatchWriterOptions(job,
-          batchConfig);
-      RecordWriter<Text,Mutation> writer = outputFormat.getRecordWriter(null, job, "Test", null);
-
-      try {
-        for (int i = 0; i < 3; i++) {
-          Mutation m = new Mutation(new Text(String.format("%08d", i)));
-          for (int j = 0; j < 3; j++) {
-            m.put("cf1", "cq" + j, i + "_" + j);
-          }
-          writer.write(new Text(testName()), m);
-        }
-
-      } catch (Exception e) {
-        e.printStackTrace();
-        // we don't want the exception to come from write
-      }
-
-      client.securityOperations().revokeTablePermission("root", testName(), TablePermission.WRITE);
-
-      var ex = assertThrows(IOException.class, () -> writer.close(null));
-      log.info(ex.getMessage(), ex);
-      assertTrue(ex.getCause() instanceof MutationsRejectedException);
-    }
-  }
-
-  private static AssertionError e1 = null;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-      OutputCollector<Text,Mutation> finalOutput;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) {
-        finalOutput = output;
-        try {
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        finalOutput.collect(new Text(), m);
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 6) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName()
-            + " <user> <pass> <inputtable> <outputtable> <instanceName> <zooKeepers>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-      String instanceName = args[4];
-      String zooKeepers = args[5];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
-
-      ClientInfo info = ClientInfo
-          .from(Accumulo.newClientProperties().to(instanceName, zooKeepers).as(user, pass).build());
-
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
-          info.getInstanceName(), info.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
-          info.getPrincipal(), info.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table1);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setZooKeeperInstance(job,
-          info.getInstanceName(), info.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setConnectorInfo(job,
-          info.getPrincipal(), info.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setCreateTables(job, false);
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setDefaultTableName(job, table2);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void testMR() throws Exception {
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-      String instanceName = getCluster().getInstanceName();
-      String table1 = instanceName + "_t1";
-      String table2 = instanceName + "_t2";
-      c.tableOperations().create(table1);
-      c.tableOperations().create(table2);
-      try (BatchWriter bw = c.createBatchWriter(table1)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-          m.put("", "", String.format("%09x", i));
-          bw.addMutation(m);
-        }
-      }
-
-      MRTester.main(new String[] {"root", ROOT_PASSWORD, table1, table2, instanceName,
-          getCluster().getZooKeepers()});
-      assertNull(e1);
-
-      try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
-            .map(Integer::parseInt).collect(onlyElement());
-        assertEquals(100, actual);
-      }
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
deleted file mode 100644
index 1142283..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
+++ /dev/null
@@ -1,208 +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.test.mapred;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyValue;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
-
-  private static final String ROW1 = "row1";
-  private static final String ROW2 = "row2";
-  private static final String ROW3 = "row3";
-  private static final String COLF1 = "colf1";
-  private static List<Entry<Key,Value>> row1;
-  private static List<Entry<Key,Value>> row2;
-  private static List<Entry<Key,Value>> row3;
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  @BeforeAll
-  public static void prepareRows() {
-    row1 = new ArrayList<>();
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
-    row2 = new ArrayList<>();
-    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
-    row3 = new ArrayList<>();
-    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
-  }
-
-  private static void checkLists(final List<Entry<Key,Value>> first,
-      final Iterator<Entry<Key,Value>> second) {
-    int entryIndex = 0;
-    while (second.hasNext()) {
-      final Entry<Key,Value> entry = second.next();
-      assertEquals(first.get(entryIndex).getKey(), entry.getKey(), "Keys should be equal");
-      assertEquals(first.get(entryIndex).getValue(), entry.getValue(), "Values should be equal");
-      entryIndex++;
-    }
-  }
-
-  private static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list)
-      throws MutationsRejectedException {
-    for (Entry<Key,Value> e : list) {
-      final Key key = e.getKey();
-      final Mutation mutation = new Mutation(key.getRow());
-      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility());
-      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility,
-          key.getTimestamp(), e.getValue());
-      writer.addMutation(mutation);
-    }
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    public static class TestMapper
-        implements Mapper<Text,PeekingIterator<Entry<Key,Value>>,Key,Value> {
-      int count = 0;
-
-      @Override
-      public void map(Text k, PeekingIterator<Entry<Key,Value>> v,
-          OutputCollector<Key,Value> output, Reporter reporter) {
-        try {
-          switch (count) {
-            case 0:
-              assertEquals(new Text(ROW1), k, "Current key should be " + ROW1);
-              checkLists(row1, v);
-              break;
-            case 1:
-              assertEquals(new Text(ROW2), k, "Current key should be " + ROW2);
-              checkLists(row2, v);
-              break;
-            case 2:
-              assertEquals(new Text(ROW3), k, "Current key should be " + ROW3);
-              checkLists(row3, v);
-              break;
-            default:
-              fail();
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() {
-        try {
-          assertEquals(3, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 1) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table>");
-      }
-
-      String table = args[0];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.setInputTableName(job, table);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-      client.tableOperations().create(tableName);
-      try (BatchWriter writer = client.createBatchWriter(tableName)) {
-        insertList(writer, row1);
-        insertList(writer, row2);
-        insertList(writer, row3);
-      }
-      MRTester.main(new String[] {tableName});
-      assertNull(e1);
-      assertNull(e2);
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
deleted file mode 100644
index 6c850fa..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
+++ /dev/null
@@ -1,208 +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.test.mapred;
-
-import static com.google.common.collect.MoreCollectors.onlyElement;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.Arrays;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.Credentials;
-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.security.Authorizations;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class TokenFileIT extends AccumuloClusterHarness {
-  private static AssertionError e1 = null;
-
-  public static class MRTokenFileTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-      OutputCollector<Text,Mutation> finalOutput;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) {
-        // verify cached token file is available locally
-        for (Class<?> formatClass : Arrays.asList(
-            org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class,
-            org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.class)) {
-          String formatName = formatClass.getSimpleName();
-          File file = new File(formatName + ".tokenfile");
-          assertTrue(file.exists());
-          assertTrue(file.canRead());
-        }
-
-        finalOutput = output;
-        try {
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        finalOutput.collect(new Text(), m);
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 3) {
-        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName()
-            + " <token file> <inputtable> <outputtable>");
-      }
-
-      String user = getAdminPrincipal();
-      String tokenFile = args[0];
-      String table1 = args[1];
-      String table2 = args[2];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
-
-      ClientInfo info = getClientInfo();
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
-          info.getInstanceName(), info.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job, user,
-          tokenFile);
-      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table1);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setZooKeeperInstance(job,
-          info.getInstanceName(), info.getZooKeepers());
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setConnectorInfo(job, user,
-          tokenFile);
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setCreateTables(job, false);
-      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setDefaultTableName(job, table2);
-
-      job.setNumReduceTasks(0);
-
-      RunningJob rj = JobClient.runJob(job);
-      if (rj.isSuccessful()) {
-        return 0;
-      } else {
-        System.out.println(rj.getFailureInfo());
-        return 1;
-      }
-    }
-
-  }
-
-  @TempDir
-  private static File tempDir;
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
-  @Test
-  public void testMR() throws Exception {
-    String[] tableNames = getUniqueNames(2);
-    String table1 = tableNames[0];
-    String table2 = tableNames[1];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table1);
-      c.tableOperations().create(table2);
-      try (BatchWriter bw = c.createBatchWriter(table1)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-          m.put("", "", String.format("%09x", i));
-          bw.addMutation(m);
-        }
-      }
-
-      File tf = new File(tempDir, "root_test.pw");
-      assertTrue(tf.createNewFile(), "Failed to create file: " + tf);
-      try (PrintStream out = new PrintStream(tf)) {
-        String outString = new Credentials(getAdminPrincipal(), getAdminToken()).serialize();
-        out.println(outString);
-      }
-
-      Configuration conf = cluster.getServerContext().getHadoopConf();
-      conf.set("hadoop.tmp.dir", new File(tf.getAbsolutePath()).getParent());
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(),
-          new String[] {tf.getAbsolutePath(), table1, table2}));
-      if (e1 != null) {
-        e1.printStackTrace();
-      }
-      assertNull(e1);
-
-      try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
-            .map(Integer::parseInt).collect(onlyElement());
-        assertEquals(100, actual);
-      }
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
deleted file mode 100644
index 606f959..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
+++ /dev/null
@@ -1,249 +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.test.mapreduce;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.io.File;
-import java.time.Duration;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.sample.RowSampler;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-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.file.FileSKVIterator;
-import org.apache.accumulo.core.file.rfile.RFileOperations;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
-
-  private String PREFIX;
-  private String BAD_TABLE;
-  private String TEST_TABLE;
-  private String EMPTY_TABLE;
-
-  private static final SamplerConfiguration SAMPLER_CONFIG =
-      new SamplerConfiguration(RowSampler.class.getName()).addOption("hasher", "murmur3_32")
-          .addOption("modulus", "3");
-
-  @TempDir
-  private static File tempDir;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(4);
-  }
-
-  @BeforeEach
-  public void setup() throws Exception {
-    PREFIX = testName() + "_";
-    BAD_TABLE = PREFIX + "_mapreduce_bad_table";
-    TEST_TABLE = PREFIX + "_mapreduce_test_table";
-    EMPTY_TABLE = PREFIX + "_mapreduce_empty_table";
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(EMPTY_TABLE);
-      c.tableOperations().create(TEST_TABLE);
-      c.tableOperations().create(BAD_TABLE);
-      try (BatchWriter bw = c.createBatchWriter(TEST_TABLE)) {
-        Mutation m = new Mutation("Key");
-        m.put("", "", "");
-        bw.addMutation(m);
-      }
-      try (BatchWriter bw = c.createBatchWriter(BAD_TABLE)) {
-        Mutation m = new Mutation("r1");
-        m.put("cf1", "cq1", "A&B");
-        m.put("cf1", "cq1", "A&B");
-        m.put("cf1", "cq2", "A&");
-        bw.addMutation(m);
-      }
-    }
-  }
-
-  @Test
-  public void testEmptyWrite() throws Exception {
-    handleWriteTests(false);
-  }
-
-  @Test
-  public void testRealWrite() throws Exception {
-    handleWriteTests(true);
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    private static class BadKeyMapper extends Mapper<Key,Value,Key,Value> {
-      int index = 0;
-
-      @Override
-      protected void map(Key key, Value value, Context context) {
-        String table = context.getConfiguration().get("MRTester_tableName");
-        assertNotNull(table);
-        try {
-          try {
-            context.write(key, value);
-            if (index == 2) {
-              fail();
-            }
-          } catch (Exception e) {
-            assertEquals(2, index);
-          }
-        } catch (AssertionError e) {
-          assertionErrors.put(table + "_map", e);
-        }
-        index++;
-      }
-
-      @Override
-      protected void cleanup(Context context) {
-        String table = context.getConfiguration().get("MRTester_tableName");
-        assertNotNull(table);
-        try {
-          assertEquals(2, index);
-        } catch (AssertionError e) {
-          assertionErrors.put(table + "_cleanup", e);
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 2) {
-        throw new IllegalArgumentException(
-            "Usage : " + MRTester.class.getName() + " <table> <outputfile>");
-      }
-
-      String table = args[0];
-      assertionErrors.put(table + "_map", new AssertionError("Dummy_map"));
-      assertionErrors.put(table + "_cleanup", new AssertionError("Dummy_cleanup"));
-
-      Job job = Job.getInstance(getConf(),
-          this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat.setOutputPath(job,
-          new Path(args[1]));
-      org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat.setSampler(job,
-          SAMPLER_CONFIG);
-
-      job.setMapperClass(
-          table.endsWith("_mapreduce_bad_table") ? BadKeyMapper.class : Mapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(
-          org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat.class);
-      job.getConfiguration().set("MRTester_tableName", table);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  private void handleWriteTests(boolean content) throws Exception {
-    File f = new File(tempDir, testName());
-    assertTrue(f.createNewFile());
-    assertTrue(f.delete());
-    MRTester.main(new String[] {content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
-
-    assertTrue(f.exists());
-    File[] files = f.listFiles(file -> file.getName().startsWith("part-m-"));
-    assertNotNull(files);
-    if (content) {
-      assertEquals(1, files.length);
-      assertTrue(files[0].exists());
-
-      Configuration conf = cluster.getServerContext().getHadoopConf();
-      DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
-      FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-          .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
-              NoCryptoServiceFactory.NONE)
-          .withTableConfiguration(acuconf).build()
-          .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
-      assertNotNull(sample);
-    } else {
-      assertEquals(0, files.length);
-    }
-  }
-
-  // track errors in the map reduce job; jobs insert a dummy error for the map and cleanup tasks (to
-  // ensure test correctness),
-  // so error tests should check to see if there is at least one error (could be more depending on
-  // the test) rather than zero
-  private static Multimap<String,AssertionError> assertionErrors = ArrayListMultimap.create();
-
-  @Test
-  public void writeBadVisibility() throws Exception {
-    File f = new File(tempDir, testName());
-    assertTrue(f.createNewFile());
-    assertTrue(f.delete());
-    MRTester.main(new String[] {BAD_TABLE, f.getAbsolutePath()});
-    assertTrue(f.exists());
-    assertEquals(1, assertionErrors.get(BAD_TABLE + "_map").size());
-    assertEquals(1, assertionErrors.get(BAD_TABLE + "_cleanup").size());
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
deleted file mode 100644
index d4af176..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
+++ /dev/null
@@ -1,557 +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.test.mapreduce;
-
-import static java.lang.System.currentTimeMillis;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.sample.RowSampler;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-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.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-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.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Level;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Multimap;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormatIT extends AccumuloClusterHarness {
-
-  org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat inputFormat;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(4);
-  }
-
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
-  }
-
-  @BeforeEach
-  public void before() {
-    inputFormat = new org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat();
-  }
-
-  /**
-   * Tests several different paths through the getSplits() method by setting different properties
-   * and verifying the results.
-   */
-  @Test
-  public void testGetSplits() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      String table = getUniqueNames(1)[0];
-      client.tableOperations().create(table);
-      insertData(client, table, currentTimeMillis());
-
-      Job job = Job.getInstance();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-
-      // split table
-      TreeSet<Text> splitsToAdd = new TreeSet<>();
-      for (int i = 0; i < 10000; i += 1000) {
-        splitsToAdd.add(new Text(String.format("%09d", i)));
-      }
-      client.tableOperations().addSplits(table, splitsToAdd);
-      sleepUninterruptibly(500, TimeUnit.MILLISECONDS); // wait for splits to be propagated
-
-      // get splits without setting any range
-      Collection<Text> actualSplits = client.tableOperations().listSplits(table);
-      List<InputSplit> splits = inputFormat.getSplits(job);
-      // No ranges set on the job so it'll start with -inf
-      assertEquals(actualSplits.size() + 1, splits.size());
-
-      // set ranges and get splits
-      List<Range> ranges = new ArrayList<>();
-      for (Text text : actualSplits) {
-        ranges.add(new Range(text));
-      }
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setRanges(job, ranges);
-      splits = inputFormat.getSplits(job);
-      assertEquals(actualSplits.size(), splits.size());
-
-      // offline mode
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setOfflineTableScan(job, true);
-      assertThrows(IOException.class, () -> inputFormat.getSplits(job));
-
-      client.tableOperations().offline(table, true);
-      splits = inputFormat.getSplits(job);
-      assertEquals(actualSplits.size(), splits.size());
-
-      // auto adjust ranges
-      ranges = new ArrayList<>();
-      for (int i = 0; i < 5; i++) {
-        // overlapping ranges
-        ranges.add(new Range(String.format("%09d", i), String.format("%09d", i + 2)));
-      }
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setRanges(job, ranges);
-      splits = inputFormat.getSplits(job);
-      assertEquals(2, splits.size());
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setAutoAdjustRanges(job, false);
-      splits = inputFormat.getSplits(job);
-      assertEquals(ranges.size(), splits.size());
-
-      // BatchScan not available for offline scans
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setBatchScan(job, true);
-      // Reset auto-adjust ranges too
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setAutoAdjustRanges(job, true);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setOfflineTableScan(job, true);
-      assertThrows(IllegalArgumentException.class, () -> inputFormat.getSplits(job));
-
-      client.tableOperations().online(table, true);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setOfflineTableScan(job, false);
-
-      // test for resumption of success
-      splits = inputFormat.getSplits(job);
-      assertEquals(2, splits.size());
-
-      // BatchScan not available with isolated iterators
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, true);
-      assertThrows(IllegalArgumentException.class, () -> inputFormat.getSplits(job));
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, false);
-
-      // test for resumption of success
-      splits = inputFormat.getSplits(job);
-      assertEquals(2, splits.size());
-
-      // BatchScan not available with local iterators
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job, true);
-      assertThrows(IllegalArgumentException.class, () -> inputFormat.getSplits(job));
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job, false);
-
-      // Check we are getting back correct type pf split
-      client.tableOperations().online(table);
-      splits = inputFormat.getSplits(job);
-      for (InputSplit split : splits) {
-        assertTrue(split instanceof org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit);
-      }
-
-      // We should divide along the tablet lines similar to when using `setAutoAdjustRanges(job,
-      // true)`
-      assertEquals(2, splits.size());
-    }
-  }
-
-  private void insertData(AccumuloClient client, String tableName, long ts)
-      throws AccumuloException, TableNotFoundException {
-    try (BatchWriter bw = client.createBatchWriter(tableName)) {
-      for (int i = 0; i < 10000; i++) {
-        String row = String.format("%09d", i);
-        Mutation m = new Mutation(new Text(row));
-        m.put(new Text("cf1"), new Text("cq1"), ts, new Value("" + i));
-        bw.addMutation(m);
-      }
-    }
-  }
-
-  // track errors in the map reduce job; jobs insert a dummy error for the map and cleanup tasks (to
-  // ensure test correctness),
-  // so error tests should check to see if there is at least one error (could be more depending on
-  // the test) rather than zero
-  private static Multimap<String,AssertionError> assertionErrors = ArrayListMultimap.create();
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) {
-        String table = context.getConfiguration().get("MRTester_tableName");
-        assertNotNull(table);
-        try {
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          assertionErrors.put(table + "_map", e);
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) {
-        String table = context.getConfiguration().get("MRTester_tableName");
-        assertNotNull(table);
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          assertionErrors.put(table + "_cleanup", e);
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 2 && args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName()
-            + " <table> <inputFormatClass> [<batchScan> <scan sample>]");
-      }
-
-      String table = args[0];
-      String inputFormatClassName = args[1];
-      boolean batchScan = false;
-      boolean sample = false;
-      if (args.length == 4) {
-        batchScan = Boolean.parseBoolean(args[2]);
-        sample = Boolean.parseBoolean(args[3]);
-      }
-
-      assertionErrors.put(table + "_map", new AssertionError("Dummy_map"));
-      assertionErrors.put(table + "_cleanup", new AssertionError("Dummy_cleanup"));
-
-      @SuppressWarnings("unchecked")
-      Class<? extends InputFormat<?,?>> inputFormatClass =
-          (Class<? extends InputFormat<?,?>>) Class.forName(inputFormatClassName);
-
-      Job job = Job.getInstance(getConf(),
-          this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-      job.getConfiguration().set("MRTester_tableName", table);
-
-      job.setInputFormatClass(inputFormatClass);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setBatchScan(job, batchScan);
-      if (sample) {
-        org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setSamplerConfiguration(job,
-            SAMPLER_CONFIG);
-      }
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static int main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      return ToolRunner.run(conf, new MRTester(), args);
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    final String TEST_TABLE_1 = getUniqueNames(1)[0];
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(TEST_TABLE_1);
-      AccumuloOutputFormatIT.insertData(c, TEST_TABLE_1);
-      assertEquals(0, MRTester.main(new String[] {TEST_TABLE_1,
-          org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName()}));
-      assertEquals(1, assertionErrors.get(TEST_TABLE_1 + "_map").size());
-      assertEquals(1, assertionErrors.get(TEST_TABLE_1 + "_cleanup").size());
-    }
-  }
-
-  private static final SamplerConfiguration SAMPLER_CONFIG =
-      new SamplerConfiguration(RowSampler.class.getName()).addOption("hasher", "murmur3_32")
-          .addOption("modulus", "3");
-
-  @Test
-  public void testSample() throws Exception {
-    final String TEST_TABLE_3 = getUniqueNames(1)[0];
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(TEST_TABLE_3,
-          new NewTableConfiguration().enableSampling(SAMPLER_CONFIG));
-      AccumuloOutputFormatIT.insertData(c, TEST_TABLE_3);
-      assertEquals(0,
-          MRTester.main(new String[] {TEST_TABLE_3,
-              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(),
-              "False", "True"}));
-      assertEquals(39, assertionErrors.get(TEST_TABLE_3 + "_map").size());
-      assertEquals(2, assertionErrors.get(TEST_TABLE_3 + "_cleanup").size());
-
-      assertionErrors.clear();
-      assertEquals(0,
-          MRTester.main(new String[] {TEST_TABLE_3,
-              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(),
-              "False", "False"}));
-      assertEquals(1, assertionErrors.get(TEST_TABLE_3 + "_map").size());
-      assertEquals(1, assertionErrors.get(TEST_TABLE_3 + "_cleanup").size());
-
-      assertionErrors.clear();
-      assertEquals(0,
-          MRTester.main(new String[] {TEST_TABLE_3,
-              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(), "True",
-              "True"}));
-      assertEquals(39, assertionErrors.get(TEST_TABLE_3 + "_map").size());
-      assertEquals(2, assertionErrors.get(TEST_TABLE_3 + "_cleanup").size());
-    }
-  }
-
-  @Test
-  public void testMapWithBatchScanner() throws Exception {
-    final String TEST_TABLE_2 = getUniqueNames(1)[0];
-
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(TEST_TABLE_2);
-      AccumuloOutputFormatIT.insertData(c, TEST_TABLE_2);
-      assertEquals(0,
-          MRTester.main(new String[] {TEST_TABLE_2,
-              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(), "True",
-              "False"}));
-      assertEquals(1, assertionErrors.get(TEST_TABLE_2 + "_map").size());
-      assertEquals(1, assertionErrors.get(TEST_TABLE_2 + "_cleanup").size());
-    }
-  }
-
-  @Test
-  public void testCorrectRangeInputSplits() throws Exception {
-    Job job = Job.getInstance();
-
-    String table = getUniqueNames(1)[0];
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns =
-        Collections.singleton(new Pair<>(new Text("foo"), new Text("bar")));
-    boolean isolated = true, localIters = true;
-    Level level = Level.WARN;
-
-    try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProps()).build()) {
-      accumuloClient.tableOperations().create(table);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanAuthorizations(job,
-          auths);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, isolated);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job,
-          localIters);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.fetchColumns(job, fetchColumns);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLogLevel(job, level);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat aif =
-          new org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat();
-
-      List<InputSplit> splits = aif.getSplits(job);
-
-      assertEquals(1, splits.size());
-
-      InputSplit split = splits.get(0);
-
-      assertEquals(org.apache.accumulo.core.client.mapreduce.RangeInputSplit.class,
-          split.getClass());
-
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit risplit =
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-
-      assertEquals(table, risplit.getTableName());
-      assertEquals(isolated, risplit.isIsolatedScan());
-      assertEquals(localIters, risplit.usesLocalIterators());
-      assertEquals(fetchColumns, risplit.getFetchedColumns());
-      assertEquals(level, risplit.getLogLevel());
-    }
-  }
-
-  @Test
-  public void testGetSplitsNoReadPermission() throws Exception {
-    Job job = Job.getInstance();
-
-    String table = getUniqueNames(1)[0];
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns =
-        Collections.singleton(new Pair<>(new Text("foo"), new Text("bar")));
-    boolean isolated = true, localIters = true;
-    Level level = Level.WARN;
-
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      client.tableOperations().create(table);
-      client.securityOperations().revokeTablePermission(client.whoami(), table,
-          TablePermission.READ);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          cluster.getClientConfig());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          getAdminPrincipal(), getAdminToken());
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanAuthorizations(job,
-          auths);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, isolated);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job,
-          localIters);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.fetchColumns(job, fetchColumns);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLogLevel(job, level);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat aif =
-          new org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat();
-
-      assertThrows(IOException.class, () -> aif.getSplits(job));
-    }
-  }
-
-  /*
-   * This tests the case where we do not have Table.READ permission, but we do have Namespace.READ.
-   * See issue #1370.
-   */
-  @Test
-  public void testGetSplitsWithNamespaceReadPermission() throws Exception {
-    Job job = Job.getInstance();
-
-    final String[] namespaceAndTable = getUniqueNames(2);
-    final String namespace = namespaceAndTable[0];
-    final String tableSimpleName = namespaceAndTable[1];
-    final String table = namespace + "." + tableSimpleName;
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns =
-        Collections.singleton(new Pair<>(new Text("foo"), new Text("bar")));
-    final boolean isolated = true;
-    final boolean localIters = true;
-    Level level = Level.WARN;
-
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      client.namespaceOperations().create(namespace); // creating namespace implies Namespace.READ
-      client.tableOperations().create(table);
-      client.securityOperations().revokeTablePermission(client.whoami(), table,
-          TablePermission.READ);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          cluster.getClientConfig());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          getAdminPrincipal(), getAdminToken());
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanAuthorizations(job,
-          auths);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, isolated);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job,
-          localIters);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.fetchColumns(job, fetchColumns);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLogLevel(job, level);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat aif =
-          new org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat();
-
-      List<InputSplit> splits = aif.getSplits(job);
-
-      assertEquals(1, splits.size());
-    }
-  }
-
-  @Test
-  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
-    String table = getUniqueNames(1)[0];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table);
-      AccumuloOutputFormatIT.insertData(c, table);
-      assertEquals(0,
-          MRTester.main(new String[] {table, EmptySplitsAccumuloInputFormat.class.getName()}));
-      assertEquals(1, assertionErrors.get(table + "_map").size());
-      assertEquals(1, assertionErrors.get(table + "_cleanup").size());
-    }
-  }
-
-  /**
-   * AccumuloInputFormat which returns an "empty" RangeInputSplit
-   */
-  public static class EmptySplitsAccumuloInputFormat
-      extends org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat {
-
-    @Override
-    public List<InputSplit> getSplits(JobContext context) throws IOException {
-      List<InputSplit> oldSplits = super.getSplits(context);
-      List<InputSplit> newSplits = new ArrayList<>(oldSplits.size());
-
-      // Copy only the necessary information
-      for (InputSplit oldSplit : oldSplits) {
-        org.apache.accumulo.core.client.mapreduce.RangeInputSplit newSplit =
-            new org.apache.accumulo.core.client.mapreduce.RangeInputSplit(
-                (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) oldSplit);
-        newSplits.add(newSplit);
-      }
-
-      return newSplits;
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
deleted file mode 100644
index e9571cc..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
+++ /dev/null
@@ -1,178 +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.test.mapreduce;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-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.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormatIT extends AccumuloClusterHarness {
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static class MRTester extends Configured implements Tool {
-
-    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) {
-        try {
-          String tableName =
-              ((org.apache.accumulo.core.client.mapreduce.RangeInputSplit) context.getInputSplit())
-                  .getTableName();
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
-          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 2) {
-        throw new IllegalArgumentException(
-            "Usage : " + MRTester.class.getName() + " <table1> <table2>");
-      }
-
-      String table1 = args[0];
-      String table2 = args[1];
-
-      Job job = Job.getInstance(getConf(),
-          this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(
-          org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat
-          .setZooKeeperInstance(job, ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig1 =
-          new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig2 =
-          new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
-
-      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configMap =
-          new HashMap<>();
-      configMap.put(table1, tableConfig1);
-      configMap.put(table2, tableConfig2);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat
-          .setInputTableConfigs(job, configMap);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  /**
-   * Generate incrementing counts and attach table name to the key/value so that order and
-   * multi-table data can be verified.
-   */
-  @Test
-  public void testMap() throws Exception {
-    String[] tableNames = getUniqueNames(2);
-    String table1 = tableNames[0];
-    String table2 = tableNames[1];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table1);
-      c.tableOperations().create(table2);
-      BatchWriter bw = c.createBatchWriter(table1);
-      BatchWriter bw2 = c.createBatchWriter(table2);
-      for (int i = 0; i < 100; i++) {
-        Mutation t1m = new Mutation(new Text(String.format("%s_%09x", table1, i + 1)));
-        t1m.put("", "", String.format("%s_%09x", table1, i));
-        bw.addMutation(t1m);
-        Mutation t2m = new Mutation(new Text(String.format("%s_%09x", table2, i + 1)));
-        t2m.put("", "", String.format("%s_%09x", table2, i));
-        bw2.addMutation(t2m);
-      }
-      bw.close();
-      bw2.close();
-
-      MRTester.main(new String[] {table1, table2});
-      assertNull(e1);
-      assertNull(e2);
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
deleted file mode 100644
index 3b94e71..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.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.test.mapreduce;
-
-import static com.google.common.collect.MoreCollectors.onlyElement;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-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.security.Authorizations;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
-  private static AssertionError e1 = null;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) {
-        try {
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        context.write(new Text(), m);
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 2) {
-        throw new IllegalArgumentException(
-            "Usage : " + MRTester.class.getName() + " <inputtable> <outputtable>");
-      }
-
-      String table1 = args[0];
-      String table2 = args[1];
-
-      Job job = Job.getInstance(getConf(),
-          this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table1);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(
-          org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setCreateTables(job, false);
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setDefaultTableName(job,
-          table2);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  public static void insertData(AccumuloClient client, String table)
-      throws TableNotFoundException, MutationsRejectedException {
-    try (BatchWriter bw = client.createBatchWriter(table)) {
-      for (int i = 0; i < 100; i++) {
-        Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-        m.put("", "", String.format("%09x", i));
-        bw.addMutation(m);
-      }
-    }
-  }
-
-  @Test
-  public void testMR() throws Exception {
-    String[] tableNames = getUniqueNames(2);
-    String table1 = tableNames[0];
-    String table2 = tableNames[1];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table1);
-      c.tableOperations().create(table2);
-      insertData(c, table1);
-
-      MRTester.main(new String[] {table1, table2});
-      assertNull(e1);
-
-      try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
-            .map(Integer::parseInt).collect(onlyElement());
-        assertEquals(100, actual);
-      }
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
deleted file mode 100644
index cbed030..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
+++ /dev/null
@@ -1,205 +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.test.mapreduce;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyValue;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
-
-  private static final String ROW1 = "row1";
-  private static final String ROW2 = "row2";
-  private static final String ROW3 = "row3";
-  private static final String COLF1 = "colf1";
-  private static List<Entry<Key,Value>> row1;
-  private static List<Entry<Key,Value>> row2;
-  private static List<Entry<Key,Value>> row3;
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  @BeforeAll
-  public static void prepareRows() {
-    row1 = new ArrayList<>();
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
-    row2 = new ArrayList<>();
-    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
-    row3 = new ArrayList<>();
-    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
-  }
-
-  private static void checkLists(final List<Entry<Key,Value>> first,
-      final Iterator<Entry<Key,Value>> second) {
-    int entryIndex = 0;
-    while (second.hasNext()) {
-      final Entry<Key,Value> entry = second.next();
-      assertEquals(first.get(entryIndex).getKey(), entry.getKey(), "Keys should be equal");
-      assertEquals(first.get(entryIndex).getValue(), entry.getValue(), "Values should be equal");
-      entryIndex++;
-    }
-  }
-
-  private static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list)
-      throws MutationsRejectedException {
-    for (Entry<Key,Value> e : list) {
-      final Key key = e.getKey();
-      final Mutation mutation = new Mutation(key.getRow());
-      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility());
-      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility,
-          key.getTimestamp(), e.getValue());
-      writer.addMutation(mutation);
-    }
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper
-        extends Mapper<Text,PeekingIterator<Entry<Key,Value>>,Key,Value> {
-      int count = 0;
-
-      @Override
-      protected void map(Text k, PeekingIterator<Entry<Key,Value>> v, Context context) {
-        try {
-          switch (count) {
-            case 0:
-              assertEquals(new Text(ROW1), k, "Current key should be " + ROW1);
-              checkLists(row1, v);
-              break;
-            case 1:
-              assertEquals(new Text(ROW2), k, "Current key should be " + ROW2);
-              checkLists(row2, v);
-              break;
-            case 2:
-              assertEquals(new Text(ROW3), k, "Current key should be " + ROW3);
-              checkLists(row3, v);
-              break;
-            default:
-              fail();
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) {
-        try {
-          assertEquals(3, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 1) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table>");
-      }
-
-      String table = args[0];
-
-      Job job = Job.getInstance(getConf(),
-          this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(
-          org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.class);
-
-      ClientInfo ci = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.setZooKeeperInstance(job,
-          ci.getInstanceName(), ci.getZooKeepers());
-      org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.setConnectorInfo(job,
-          ci.getPrincipal(), ci.getAuthenticationToken());
-      org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.setInputTableName(job,
-          table);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      String tableName = getUniqueNames(1)[0];
-      client.tableOperations().create(tableName);
-      try (BatchWriter writer = client.createBatchWriter(tableName)) {
-        insertList(writer, row1);
-        insertList(writer, row2);
-        insertList(writer, row3);
-      }
-      MRTester.main(new String[] {tableName});
-      assertNull(e1);
-      assertNull(e2);
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.java
deleted file mode 100644
index 0ee0898..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.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.test.mapreduce;
-
-import java.io.IOException;
-import java.util.Base64;
-import java.util.Collections;
-
-import org.apache.accumulo.core.cli.ClientOpts;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientConfConverter;
-import org.apache.accumulo.core.conf.ClientProperty;
-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.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import com.beust.jcommander.Parameter;
-
-/**
- * This class supports deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class RowHash extends Configured implements Tool {
-  /**
-   * The Mapper class that given a row number, will generate the appropriate output line.
-   */
-  public static class HashDataMapper extends Mapper<Key,Value,Text,Mutation> {
-    @Override
-    public void map(Key row, Value data, Context context) throws IOException, InterruptedException {
-      Mutation m = new Mutation(row.getRow());
-      m.put(new Text("cf-HASHTYPE"), new Text("cq-MD5BASE64"),
-          new Value(Base64.getEncoder().encode(MD5Hash.digest(data.toString()).getDigest())));
-      context.write(null, m);
-      context.progress();
-    }
-
-    @Override
-    public void setup(Context job) {}
-  }
-
-  private static class Opts extends ClientOpts {
-    @Parameter(names = "--column", required = true)
-    String column;
-
-    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
-    String tableName;
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    job.setJobName(this.getClass().getName());
-    job.setJarByClass(this.getClass());
-    Opts opts = new Opts();
-    opts.parseArgs(RowHash.class.getName(), args);
-
-    job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        ClientConfConverter.toClientConf(opts.getClientProps());
-    org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-        clientConf);
-    org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setZooKeeperInstance(job,
-        clientConf);
-
-    final String principal = ClientProperty.AUTH_PRINCIPAL.getValue(opts.getClientProps());
-    AuthenticationToken token = opts.getToken();
-    org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job, principal,
-        token);
-    org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setConnectorInfo(job, principal,
-        token);
-    org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job,
-        opts.tableName);
-    org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanAuthorizations(job,
-        opts.auths);
-    org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setCreateTables(job, true);
-    org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setDefaultTableName(job,
-        opts.tableName);
-
-    String col = opts.column;
-    int idx = col.indexOf(":");
-    Text cf = new Text(idx < 0 ? col : col.substring(0, idx));
-    Text cq = idx < 0 ? null : new Text(col.substring(idx + 1));
-    if (cf.getLength() > 0) {
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.fetchColumns(job,
-          Collections.singleton(new Pair<>(cf, cq)));
-    }
-
-    job.setMapperClass(HashDataMapper.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(Mutation.class);
-
-    job.setNumReduceTasks(0);
-
-    job.setOutputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class);
-
-    job.waitForCompletion(true);
-    return job.isSuccessful() ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    ToolRunner.run(new Configuration(), new RowHash(), args);
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
deleted file mode 100644
index 07ed91b..0000000
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.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.test.mapreduce;
-
-import static com.google.common.collect.MoreCollectors.onlyElement;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.Credentials;
-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.security.Authorizations;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * This tests deprecated mapreduce code in core jar
- */
-@Deprecated(since = "2.0.0")
-public class TokenFileIT extends AccumuloClusterHarness {
-  private static AssertionError e1 = null;
-
-  private static class MRTokenFileTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) {
-        try {
-          // verify cached token file is available locally
-          URI[] cachedFiles;
-          try {
-            cachedFiles = context.getCacheFiles();
-          } catch (IOException e) {
-            throw new AssertionError("IOException getting cache files", e);
-          }
-          assertEquals(2, cachedFiles.length); // one for each in/out format
-          for (Class<?> formatClass : Arrays.asList(
-              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class,
-              org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class)) {
-            String formatName = formatClass.getSimpleName();
-            assertTrue(Arrays.stream(cachedFiles)
-                .anyMatch(uri -> uri.toString().endsWith(formatName + ".tokenfile")));
-            File file = new File(formatName + ".tokenfile");
-            assertTrue(file.exists());
-            assertTrue(file.canRead());
-          }
-
-          if (key != null) {
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          }
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        context.write(new Text(), m);
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 3) {
-        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName()
-            + " <token file> <inputtable> <outputtable>");
-      }
-
-      String user = getAdminPrincipal();
-      String tokenFile = args[0];
-      String table1 = args[1];
-      String table2 = args[2];
-
-      Job job = Job.getInstance(getConf(),
-          this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
-
-      ClientInfo info = getClientInfo();
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job, user,
-          tokenFile);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table1);
-      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
-          info.getInstanceName(), info.getZooKeepers());
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(
-          org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setConnectorInfo(job, user,
-          tokenFile);
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setCreateTables(job, false);
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setDefaultTableName(job,
-          table2);
-      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setZooKeeperInstance(job,
-          info.getInstanceName(), info.getZooKeepers());
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      if (job.isSuccessful()) {
-        return 0;
-      } else {
-        System.out.println(job.getStatus().getFailureInfo());
-        return 1;
-      }
-    }
-  }
-
-  @TempDir
-  private static File tempDir;
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
-  @Test
-  public void testMR() throws Exception {
-    String[] tableNames = getUniqueNames(2);
-    String table1 = tableNames[0];
-    String table2 = tableNames[1];
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
-      c.tableOperations().create(table1);
-      c.tableOperations().create(table2);
-      try (BatchWriter bw = c.createBatchWriter(table1)) {
-        for (int i = 0; i < 100; i++) {
-          Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-          m.put("", "", String.format("%09x", i));
-          bw.addMutation(m);
-        }
-      }
-
-      File tf = new File(tempDir, "root_test.pw");
-      assertTrue(tf.createNewFile(), "Failed to create file: " + tf);
-      try (PrintStream out = new PrintStream(tf)) {
-        String outString = new Credentials(getAdminPrincipal(), getAdminToken()).serialize();
-        out.println(outString);
-      }
-
-      Configuration conf = cluster.getServerContext().getHadoopConf();
-      conf.set("hadoop.tmp.dir", new File(tf.getAbsolutePath()).getParent());
-      conf.set("mapreduce.framework.name", "local");
-      conf.set("mapreduce.cluster.local.dir",
-          new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(),
-          new String[] {tf.getAbsolutePath(), table1, table2}));
-      if (e1 != null) {
-        e1.printStackTrace();
-      }
-      assertNull(e1);
-
-      try (Scanner scanner = c.createScanner(table2, new Authorizations())) {
-        int actual = scanner.stream().map(Entry::getValue).map(Value::get).map(String::new)
-            .map(Integer::parseInt).collect(onlyElement());
-        assertEquals(100, actual);
-      }
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
index 55622d0..f2be6e7 100644
--- a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java
@@ -99,9 +99,10 @@
     doWorkToGenerateMetrics();
     cluster.stop();
 
-    Set<String> unexpectedMetrics = Set.of(METRICS_SCAN_YIELDS, METRICS_UPDATE_ERRORS,
-        METRICS_REPLICATION_QUEUE, METRICS_COMPACTOR_MAJC_STUCK, METRICS_SCAN_BUSY_TIMEOUT_COUNTER);
-    // add sserver as flaky until scan server included in mini tests.
+    Set<String> unexpectedMetrics =
+        Set.of(METRICS_SCAN_YIELDS, METRICS_UPDATE_ERRORS, METRICS_COMPACTOR_MAJC_STUCK,
+            METRICS_SCAN_BUSY_TIMEOUT_COUNTER, METRICS_SCAN_PAUSED_FOR_MEM,
+            METRICS_SCAN_RETURN_FOR_MEM, METRICS_MINC_PAUSED, METRICS_MAJC_PAUSED);
     Set<String> flakyMetrics = Set.of(METRICS_GC_WAL_ERRORS, METRICS_FATE_TYPE_IN_PROGRESS,
         METRICS_SCAN_BUSY_TIMEOUT_COUNTER, METRICS_SCAN_RESERVATION_TIMER,
         METRICS_SCAN_TABLET_METADATA_CACHE);
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index 67a689c..8e5a96c 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.performance;
 
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
@@ -26,10 +26,10 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.clientImpl.thrift.ClientService;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -40,7 +40,6 @@
 import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
 import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
-import org.apache.accumulo.core.dataImpl.thrift.MapFileInfo;
 import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
 import org.apache.accumulo.core.dataImpl.thrift.TCMResult;
@@ -54,25 +53,26 @@
 import org.apache.accumulo.core.dataImpl.thrift.TSummaries;
 import org.apache.accumulo.core.dataImpl.thrift.TSummaryRequest;
 import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.manager.thrift.TabletServerStatus;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
 import org.apache.accumulo.core.metadata.TabletLocationState;
 import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
+import org.apache.accumulo.core.tabletingest.thrift.DataFileInfo;
+import org.apache.accumulo.core.tabletingest.thrift.TDurability;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
+import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
-import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
-import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
-import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.client.ClientServiceHandler;
@@ -87,6 +87,7 @@
 import org.apache.thrift.TMultiplexedProcessor;
 
 import com.beust.jcommander.Parameter;
+import com.google.common.net.HostAndPort;
 
 /**
  * The purpose of this class is to server as fake tserver that is a data sink like /dev/null.
@@ -96,7 +97,8 @@
 public class NullTserver {
 
   public static class NullTServerTabletClientHandler
-      implements TabletClientService.Iface, TabletScanClientService.Iface {
+      implements TabletServerClientService.Iface, TabletScanClientService.Iface,
+      TabletIngestClientService.Iface, TabletManagementClientService.Iface {
 
     private long updateSession = 1;
 
@@ -120,14 +122,8 @@
     }
 
     @Override
-    public List<TKeyExtent> bulkImport(TInfo tinfo, TCredentials credentials, long tid,
-        Map<TKeyExtent,Map<String,MapFileInfo>> files, boolean setTime) {
-      return null;
-    }
-
-    @Override
     public void loadFiles(TInfo tinfo, TCredentials credentials, long tid, String dir,
-        Map<TKeyExtent,Map<String,MapFileInfo>> fileMap, boolean setTime) {}
+        Map<TKeyExtent,Map<String,DataFileInfo>> fileMap, boolean setTime) {}
 
     @Override
     public void closeMultiScan(TInfo tinfo, long scanID) {}
@@ -170,12 +166,6 @@
     }
 
     @Override
-    public void update(TInfo tinfo, TCredentials credentials, TKeyExtent keyExtent,
-        TMutation mutation, TDurability durability) {
-
-    }
-
-    @Override
     public TabletServerStatus getTabletServerStatus(TInfo tinfo, TCredentials credentials) {
       return null;
     }
@@ -209,9 +199,6 @@
     }
 
     @Override
-    public void chop(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent extent) {}
-
-    @Override
     public void flushTablet(TInfo tinfo, TCredentials credentials, String lock, TKeyExtent extent) {
 
     }
@@ -336,12 +323,18 @@
         ThriftProcessorTypes.CLIENT.getTProcessor(ClientService.Processor.class,
             ClientService.Iface.class, csh, context));
     muxProcessor.registerProcessor(ThriftClientTypes.TABLET_SERVER.getServiceName(),
-        ThriftProcessorTypes.TABLET_SERVER.getTProcessor(TabletClientService.Processor.class,
-            TabletClientService.Iface.class, tch, context));
-    muxProcessor.registerProcessor(ThriftProcessorTypes.TABLET_SERVER_SCAN.getServiceName(),
-        ThriftProcessorTypes.TABLET_SERVER_SCAN.getTProcessor(
-            TabletScanClientService.Processor.class, TabletScanClientService.Iface.class, tch,
-            context));
+        ThriftProcessorTypes.TABLET_SERVER.getTProcessor(TabletServerClientService.Processor.class,
+            TabletServerClientService.Iface.class, tch, context));
+    muxProcessor.registerProcessor(ThriftProcessorTypes.TABLET_SCAN.getServiceName(),
+        ThriftProcessorTypes.TABLET_SCAN.getTProcessor(TabletScanClientService.Processor.class,
+            TabletScanClientService.Iface.class, tch, context));
+    muxProcessor.registerProcessor(ThriftClientTypes.TABLET_INGEST.getServiceName(),
+        ThriftProcessorTypes.TABLET_INGEST.getTProcessor(TabletIngestClientService.Processor.class,
+            TabletIngestClientService.Iface.class, tch, context));
+    muxProcessor.registerProcessor(ThriftProcessorTypes.TABLET_MGMT.getServiceName(),
+        ThriftProcessorTypes.TABLET_MGMT.getTProcessor(
+            TabletManagementClientService.Processor.class,
+            TabletManagementClientService.Iface.class, tch, context));
 
     TServerUtils.startTServer(context.getConfiguration(), ThriftServerType.CUSTOM_HS_HA,
         muxProcessor, "NullTServer", "null tserver", 2, ThreadPools.DEFAULT_TIMEOUT_MILLISECS, 1000,
@@ -355,7 +348,8 @@
     // read the locations for the table
     Range tableRange = new KeyExtent(tableId, null, null).toMetaRange();
     List<Assignment> assignments = new ArrayList<>();
-    try (var s = new MetaDataTableScanner(context, tableRange, MetadataTable.NAME)) {
+    try (
+        var s = new MetaDataTableScanner(context, tableRange, AccumuloTable.METADATA.tableName())) {
       long randomSessionID = opts.port;
       TServerInstance instance = new TServerInstance(addr, randomSessionID);
 
@@ -369,7 +363,7 @@
     store.setLocations(assignments);
 
     while (true) {
-      sleepUninterruptibly(10, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(10));
     }
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index 63ec7fc..4655b06 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.performance.scan;
 
-import static org.apache.accumulo.harness.AccumuloITBase.random;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
 import java.net.InetAddress;
@@ -64,10 +64,10 @@
 import org.apache.accumulo.core.iteratorsImpl.system.SortedMapIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.VisibilityFilter;
 import org.apache.accumulo.core.metadata.MetadataServicer;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Stat;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.cli.ServerUtilOpts;
@@ -77,12 +77,12 @@
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.Parameter;
+import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
@@ -137,10 +137,10 @@
 
     List<KeyExtent> tabletsToTest = selectRandomTablets(opts.numThreads, candidates);
 
-    Map<KeyExtent,List<TabletFile>> tabletFiles = new HashMap<>();
+    Map<KeyExtent,List<StoredTabletFile>> tabletFiles = new HashMap<>();
 
     for (KeyExtent ke : tabletsToTest) {
-      List<TabletFile> files = getTabletFiles(context, ke);
+      List<StoredTabletFile> files = getTabletFiles(context, ke);
       tabletFiles.put(ke, files);
     }
 
@@ -167,7 +167,7 @@
       ArrayList<Test> tests = new ArrayList<>();
 
       for (final KeyExtent ke : tabletsToTest) {
-        final List<TabletFile> files = tabletFiles.get(ke);
+        final List<StoredTabletFile> files = tabletFiles.get(ke);
         Test test = new Test(ke) {
           @Override
           public int runTest() throws Exception {
@@ -187,7 +187,7 @@
       ArrayList<Test> tests = new ArrayList<>();
 
       for (final KeyExtent ke : tabletsToTest) {
-        final List<TabletFile> files = tabletFiles.get(ke);
+        final List<StoredTabletFile> files = tabletFiles.get(ke);
         Test test = new Test(ke) {
           @Override
           public int runTest() throws Exception {
@@ -205,7 +205,7 @@
       ArrayList<Test> tests = new ArrayList<>();
 
       for (final KeyExtent ke : tabletsToTest) {
-        final List<TabletFile> files = tabletFiles.get(ke);
+        final List<StoredTabletFile> files = tabletFiles.get(ke);
         Test test = new Test(ke) {
           @Override
           public int runTest() throws Exception {
@@ -382,7 +382,7 @@
     List<KeyExtent> tabletsToTest = new ArrayList<>();
 
     for (int i = 0; i < numThreads; i++) {
-      int rindex = random.nextInt(candidates.size());
+      int rindex = RANDOM.get().nextInt(candidates.size());
       tabletsToTest.add(candidates.get(rindex));
       Collections.swap(candidates, rindex, candidates.size() - 1);
       candidates = candidates.subList(0, candidates.size() - 1);
@@ -390,14 +390,14 @@
     return tabletsToTest;
   }
 
-  private static List<TabletFile> getTabletFiles(ServerContext context, KeyExtent ke)
+  private static List<StoredTabletFile> getTabletFiles(ServerContext context, KeyExtent ke)
       throws IOException {
     return new ArrayList<>(
         MetadataTableUtil.getFileAndLogEntries(context, ke).getSecond().keySet());
   }
 
-  private static void reportHdfsBlockLocations(ServerContext context, List<TabletFile> files)
-      throws Exception {
+  private static void reportHdfsBlockLocations(ServerContext context,
+      List<? extends TabletFile> files) throws Exception {
     VolumeManager fs = context.getVolumeManager();
 
     System.out.println("\t\tFile block report : ");
@@ -405,8 +405,8 @@
       FileStatus status = fs.getFileStatus(file.getPath());
 
       if (status.isDirectory()) {
-        // assume it is a map file
-        status = fs.getFileStatus(new Path(file + "/data"));
+        log.warn("Saw unexpected directory at {} while getting block locations", file);
+        continue;
       }
       FileSystem ns = fs.getFileSystemByPath(file.getPath());
       BlockLocation[] locs = ns.getFileBlockLocations(status, 0, status.getLen());
@@ -427,16 +427,16 @@
   }
 
   private static SortedKeyValueIterator<Key,Value> createScanIterator(KeyExtent ke,
-      Collection<SortedKeyValueIterator<Key,Value>> mapfiles, Authorizations authorizations,
+      Collection<SortedKeyValueIterator<Key,Value>> dataFiles, Authorizations authorizations,
       byte[] defaultLabels, HashSet<Column> columnSet, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, boolean useTableIterators, TableConfiguration conf)
       throws IOException {
 
     SortedMapIterator smi = new SortedMapIterator(new TreeMap<>());
 
-    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(mapfiles.size() + 1);
+    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(dataFiles.size() + 1);
 
-    iters.addAll(mapfiles);
+    iters.addAll(dataFiles);
     iters.add(smi);
 
     MultiIterator multiIter = new MultiIterator(iters, ke);
@@ -456,16 +456,16 @@
   }
 
   private static int readFiles(VolumeManager fs, AccumuloConfiguration aconf,
-      List<TabletFile> files, KeyExtent ke, String[] columns) throws Exception {
+      List<StoredTabletFile> files, KeyExtent ke, String[] columns) throws Exception {
 
     int count = 0;
 
     HashSet<ByteSequence> columnSet = createColumnBSS(columns);
 
-    for (TabletFile file : files) {
+    for (StoredTabletFile file : files) {
       FileSystem ns = fs.getFileSystemByPath(file.getPath());
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.getPathStr(), ns, ns.getConf(), NoCryptoServiceFactory.NONE)
+          .forFile(file, ns, ns.getConf(), NoCryptoServiceFactory.NONE)
           .withTableConfiguration(aconf).build();
       Range range = new Range(ke.prevEndRow(), false, ke.endRow(), true);
       reader.seek(range, columnSet, !columnSet.isEmpty());
@@ -488,17 +488,17 @@
   }
 
   private static int readFilesUsingIterStack(VolumeManager fs, ServerContext context,
-      List<TabletFile> files, Authorizations auths, KeyExtent ke, String[] columns,
+      List<StoredTabletFile> files, Authorizations auths, KeyExtent ke, String[] columns,
       boolean useTableIterators) throws Exception {
 
     SortedKeyValueIterator<Key,Value> reader;
 
     List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<>(files.size());
 
-    for (TabletFile file : files) {
+    for (StoredTabletFile file : files) {
       FileSystem ns = fs.getFileSystemByPath(file.getPath());
       readers.add(FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.getPathStr(), ns, ns.getConf(), NoCryptoServiceFactory.NONE)
+          .forFile(file, ns, ns.getConf(), NoCryptoServiceFactory.NONE)
           .withTableConfiguration(context.getConfiguration()).build());
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
deleted file mode 100644
index 2db8808..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ /dev/null
@@ -1,379 +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.test.replication;
-
-import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.OutputStream;
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
-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.iterators.LongCombiner.Type;
-import org.apache.accumulo.core.iterators.user.SummingCombiner;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.harness.AccumuloITBase;
-import org.apache.accumulo.harness.Timeout;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.miniclusterImpl.ProcessReference;
-import org.apache.accumulo.miniclusterImpl.ZooKeeperBindException;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Tag;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.RegisterExtension;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-@SuppressWarnings("deprecation")
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Tag(MINI_CLUSTER_ONLY)
-@Deprecated
-public class CyclicReplicationIT extends AccumuloITBase {
-  private static final Logger log = LoggerFactory.getLogger(CyclicReplicationIT.class);
-
-  @RegisterExtension
-  Timeout timeout = Timeout.from(() -> {
-    long waitLonger = 1L;
-    try {
-      String timeoutString = System.getProperty("timeout.factor");
-      if (timeoutString != null && !timeoutString.isEmpty()) {
-        waitLonger = Long.parseLong(timeoutString);
-      }
-    } catch (NumberFormatException exception) {
-      log.warn("Could not parse timeout.factor, not scaling timeout");
-    }
-
-    return Duration.ofMinutes(waitLonger * 10);
-  });
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
-  private File createTheTestDir(String name) {
-    File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests");
-    assertTrue(baseDir.mkdirs() || baseDir.isDirectory());
-    File testDir = new File(baseDir, this.getClass().getName() + "_" + testName() + "_" + name);
-    FileUtils.deleteQuietly(testDir);
-    assertTrue(testDir.mkdir());
-    return testDir;
-  }
-
-  private void setCoreSite(MiniAccumuloClusterImpl cluster) throws Exception {
-    File csFile = new File(cluster.getConfig().getConfDir(), "core-site.xml");
-    if (csFile.exists()) {
-      throw new RuntimeException(csFile + " already exist");
-    }
-
-    Configuration coreSite = new Configuration(false);
-    coreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-    OutputStream out = new BufferedOutputStream(
-        new FileOutputStream(new File(cluster.getConfig().getConfDir(), "core-site.xml")));
-    coreSite.writeXml(out);
-    out.close();
-  }
-
-  /**
-   * Use the same SSL and credential provider configuration that is set up by AbstractMacIT for the
-   * other MAC used for replication
-   */
-  private void updatePeerConfigFromPrimary(MiniAccumuloConfigImpl primaryCfg,
-      MiniAccumuloConfigImpl peerCfg) {
-    // Set the same SSL information from the primary when present
-    Map<String,String> primarySiteConfig = primaryCfg.getSiteConfig();
-    if ("true".equals(primarySiteConfig.get(Property.INSTANCE_RPC_SSL_ENABLED.getKey()))) {
-      Map<String,String> peerSiteConfig = new HashMap<>();
-      peerSiteConfig.put(Property.INSTANCE_RPC_SSL_ENABLED.getKey(), "true");
-      String keystorePath = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey());
-      assertNotNull(keystorePath, "Keystore Path was null");
-      peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PATH.getKey(), keystorePath);
-      String truststorePath = primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey());
-      assertNotNull(truststorePath, "Truststore Path was null");
-      peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PATH.getKey(), truststorePath);
-
-      // Passwords might be stored in CredentialProvider
-      String keystorePassword = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey());
-      if (keystorePassword != null) {
-        peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey(), keystorePassword);
-      }
-      String truststorePassword =
-          primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey());
-      if (truststorePassword != null) {
-        peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
-      }
-
-      System.out.println("Setting site configuration for peer " + peerSiteConfig);
-      peerCfg.setSiteConfig(peerSiteConfig);
-    }
-
-    // Use the CredentialProvider if the primary also uses one
-    String credProvider =
-        primarySiteConfig.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
-    if (credProvider != null) {
-      Map<String,String> peerSiteConfig = peerCfg.getSiteConfig();
-      peerSiteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(),
-          credProvider);
-      peerCfg.setSiteConfig(peerSiteConfig);
-    }
-  }
-
-  @Test
-  public void dataIsNotOverReplicated() throws Exception {
-    File manager1Dir = createTheTestDir("manager1"), manager2Dir = createTheTestDir("manager2");
-    String password = "password";
-
-    MiniAccumuloConfigImpl manager1Cfg;
-    MiniAccumuloClusterImpl manager1Cluster;
-    while (true) {
-      manager1Cfg = new MiniAccumuloConfigImpl(manager1Dir, password);
-      manager1Cfg.setNumTservers(1);
-      manager1Cfg.setInstanceName("manager1");
-
-      // Set up SSL if needed
-      ConfigurableMacBase.configureForEnvironment(manager1Cfg,
-          ConfigurableMacBase.getSslDir(manager1Dir));
-
-      manager1Cfg.setProperty(Property.REPLICATION_NAME, manager1Cfg.getInstanceName());
-      manager1Cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "5M");
-      manager1Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
-      manager1Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-      manager1Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-      manager1Cluster = new MiniAccumuloClusterImpl(manager1Cfg);
-      setCoreSite(manager1Cluster);
-
-      try {
-        manager1Cluster.start();
-        break;
-      } catch (ZooKeeperBindException e) {
-        log.warn("Failed to start ZooKeeper on {}, will retry", manager1Cfg.getZooKeeperPort());
-      }
-    }
-
-    MiniAccumuloConfigImpl manager2Cfg;
-    MiniAccumuloClusterImpl manager2Cluster;
-    while (true) {
-      manager2Cfg = new MiniAccumuloConfigImpl(manager2Dir, password);
-      manager2Cfg.setNumTservers(1);
-      manager2Cfg.setInstanceName("manager2");
-
-      // Set up SSL if needed. Need to share the same SSL truststore as manager1
-      this.updatePeerConfigFromPrimary(manager1Cfg, manager2Cfg);
-
-      manager2Cfg.setProperty(Property.REPLICATION_NAME, manager2Cfg.getInstanceName());
-      manager2Cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "5M");
-      manager2Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
-      manager2Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-      manager2Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-      manager2Cluster = new MiniAccumuloClusterImpl(manager2Cfg);
-      setCoreSite(manager2Cluster);
-
-      try {
-        manager2Cluster.start();
-        break;
-      } catch (ZooKeeperBindException e) {
-        log.warn("Failed to start ZooKeeper on {}, will retry", manager2Cfg.getZooKeeperPort());
-      }
-    }
-
-    try {
-      AccumuloClient clientManager1 =
-          manager1Cluster.createAccumuloClient("root", new PasswordToken(password)),
-          clientManager2 =
-              manager2Cluster.createAccumuloClient("root", new PasswordToken(password));
-
-      String manager1UserName = "manager1", manager1Password = "foo";
-      String manager2UserName = "manager2", manager2Password = "bar";
-      String manager1Table = manager1Cluster.getInstanceName(),
-          manager2Table = manager2Cluster.getInstanceName();
-
-      clientManager1.securityOperations().createLocalUser(manager1UserName,
-          new PasswordToken(manager1Password));
-      clientManager2.securityOperations().createLocalUser(manager2UserName,
-          new PasswordToken(manager2Password));
-
-      // Configure the credentials we should use to authenticate ourselves to the peer for
-      // replication
-      clientManager1.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_USER.getKey() + manager2Cluster.getInstanceName(),
-          manager2UserName);
-      clientManager1.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_PASSWORD.getKey() + manager2Cluster.getInstanceName(),
-          manager2Password);
-
-      clientManager2.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_USER.getKey() + manager1Cluster.getInstanceName(),
-          manager1UserName);
-      clientManager2.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_PASSWORD.getKey() + manager1Cluster.getInstanceName(),
-          manager1Password);
-
-      clientManager1.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + manager2Cluster.getInstanceName(),
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(manager2Cluster.getInstanceName(),
-                  manager2Cluster.getZooKeepers())));
-
-      clientManager2.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + manager1Cluster.getInstanceName(),
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(manager1Cluster.getInstanceName(),
-                  manager1Cluster.getZooKeepers())));
-
-      clientManager1.tableOperations().create(manager1Table,
-          new NewTableConfiguration().withoutDefaultIterators());
-      String manager1TableId = clientManager1.tableOperations().tableIdMap().get(manager1Table);
-      assertNotNull(manager1TableId);
-
-      clientManager2.tableOperations().create(manager2Table,
-          new NewTableConfiguration().withoutDefaultIterators());
-      String manager2TableId = clientManager2.tableOperations().tableIdMap().get(manager2Table);
-      assertNotNull(manager2TableId);
-
-      // Replicate manager1 in the manager1 cluster to manager2 in the manager2 cluster
-      clientManager1.tableOperations().setProperty(manager1Table,
-          Property.TABLE_REPLICATION.getKey(), "true");
-      clientManager1.tableOperations().setProperty(manager1Table,
-          Property.TABLE_REPLICATION_TARGET.getKey() + manager2Cluster.getInstanceName(),
-          manager2TableId);
-
-      // Replicate manager2 in the manager2 cluster to manager1 in the manager2 cluster
-      clientManager2.tableOperations().setProperty(manager2Table,
-          Property.TABLE_REPLICATION.getKey(), "true");
-      clientManager2.tableOperations().setProperty(manager2Table,
-          Property.TABLE_REPLICATION_TARGET.getKey() + manager1Cluster.getInstanceName(),
-          manager1TableId);
-
-      // Give our replication user the ability to write to the respective table
-      clientManager1.securityOperations().grantTablePermission(manager1UserName, manager1Table,
-          TablePermission.WRITE);
-      clientManager2.securityOperations().grantTablePermission(manager2UserName, manager2Table,
-          TablePermission.WRITE);
-
-      IteratorSetting summingCombiner = new IteratorSetting(50, SummingCombiner.class);
-      SummingCombiner.setEncodingType(summingCombiner, Type.STRING);
-      SummingCombiner.setCombineAllColumns(summingCombiner, true);
-
-      // Set a combiner on both instances that will sum multiple values
-      // We can use this to verify that the mutation was not sent multiple times
-      clientManager1.tableOperations().attachIterator(manager1Table, summingCombiner);
-      clientManager2.tableOperations().attachIterator(manager2Table, summingCombiner);
-
-      // Write a single entry
-      try (BatchWriter bw = clientManager1.createBatchWriter(manager1Table)) {
-        Mutation m = new Mutation("row");
-        m.put("count", "", "1");
-        bw.addMutation(m);
-      }
-
-      Set<String> files = clientManager1.replicationOperations().referencedFiles(manager1Table);
-
-      log.info("Found {} that need replication from manager1", files);
-
-      // Kill and restart the tserver to close the WAL on manager1
-      for (ProcessReference proc : manager1Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        manager1Cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-
-      manager1Cluster.exec(TabletServer.class);
-
-      log.info("Restarted tserver on manager1");
-
-      // Try to avoid ACCUMULO-2964
-      Thread.sleep(1000);
-
-      // Sanity check that the element is there on manager1
-      Entry<Key,Value> entry;
-      try (Scanner s = clientManager1.createScanner(manager1Table, Authorizations.EMPTY)) {
-        entry = getOnlyElement(s);
-        assertEquals("1", entry.getValue().toString());
-
-        // Wait for this table to replicate
-        clientManager1.replicationOperations().drain(manager1Table, files);
-
-        Thread.sleep(5000);
-      }
-
-      // Check that the element made it to manager2 only once
-      try (Scanner s = clientManager2.createScanner(manager2Table, Authorizations.EMPTY)) {
-        entry = getOnlyElement(s);
-        assertEquals("1", entry.getValue().toString());
-
-        // Wait for manager2 to finish replicating it back
-        files = clientManager2.replicationOperations().referencedFiles(manager2Table);
-
-        // Kill and restart the tserver to close the WAL on manager2
-        for (ProcessReference proc : manager2Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-          manager2Cluster.killProcess(ServerType.TABLET_SERVER, proc);
-        }
-
-        manager2Cluster.exec(TabletServer.class);
-
-        // Try to avoid ACCUMULO-2964
-        Thread.sleep(1000);
-      }
-
-      // Check that the element made it to manager2 only once
-      try (Scanner s = clientManager2.createScanner(manager2Table, Authorizations.EMPTY)) {
-        entry = getOnlyElement(s);
-        assertEquals("1", entry.getValue().toString());
-
-        clientManager2.replicationOperations().drain(manager2Table, files);
-
-        Thread.sleep(5000);
-      }
-
-      // Verify that the entry wasn't sent back to manager1
-      try (Scanner s = clientManager1.createScanner(manager1Table, Authorizations.EMPTY)) {
-        entry = getOnlyElement(s);
-        assertEquals("1", entry.getValue().toString());
-      }
-    } finally {
-      manager1Cluster.stop();
-      manager2Cluster.stop();
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java b/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
deleted file mode 100644
index 59a53cbc..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
+++ /dev/null
@@ -1,193 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Map.Entry;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-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.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.FinishedWorkUpdater;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class FinishedWorkUpdaterIT extends ConfigurableMacBase {
-
-  private AccumuloClient client;
-  private FinishedWorkUpdater updater;
-
-  @BeforeEach
-  public void configureUpdater() {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    updater = new FinishedWorkUpdater(client);
-  }
-
-  @Test
-  public void offlineReplicationTableFailsGracefully() {
-    updater.run();
-  }
-
-  @Test
-  public void recordsWithProgressUpdateBothTables() throws Exception {
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    ReplicationTable.setOnline(client);
-
-    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat =
-        Status.newBuilder().setBegin(100).setEnd(200).setClosed(true).setInfiniteEnd(false).build();
-    ReplicationTarget target = new ReplicationTarget("peer", "table1", TableId.of("1"));
-
-    // Create a single work record for a file to some peer
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    Mutation m = new Mutation(file);
-    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    bw.close();
-
-    updater.run();
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      s.setRange(Range.exact(file));
-      StatusSection.limit(s);
-      Entry<Key,Value> entry = getOnlyElement(s);
-
-      assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
-      assertEquals(entry.getKey().getColumnQualifier().toString(),
-          target.getSourceTableId().canonical());
-
-      // We should only rely on the correct begin attribute being returned
-      Status actual = Status.parseFrom(entry.getValue().get());
-      assertEquals(stat.getBegin(), actual.getBegin());
-    }
-  }
-
-  @Test
-  public void chooseMinimumBeginOffset() throws Exception {
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    ReplicationTable.setOnline(client);
-
-    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true)
-        .setInfiniteEnd(false).build();
-    Status stat2 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true)
-        .setInfiniteEnd(false).build();
-    Status stat3 =
-        Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(false).build();
-    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", TableId.of("1"));
-    ReplicationTarget target2 = new ReplicationTarget("peer2", "table2", TableId.of("1"));
-    ReplicationTarget target3 = new ReplicationTarget("peer3", "table3", TableId.of("1"));
-
-    // Create a single work record for a file to some peer
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    Mutation m = new Mutation(file);
-    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
-    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
-    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
-    bw.addMutation(m);
-    bw.close();
-
-    updater.run();
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      s.setRange(Range.exact(file));
-      StatusSection.limit(s);
-      Entry<Key,Value> entry = getOnlyElement(s);
-
-      assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
-      assertEquals(entry.getKey().getColumnQualifier().toString(),
-          target1.getSourceTableId().canonical());
-
-      // We should only rely on the correct begin attribute being returned
-      Status actual = Status.parseFrom(entry.getValue().get());
-      assertEquals(1, actual.getBegin());
-    }
-  }
-
-  @Test
-  public void chooseMinimumBeginOffsetInfiniteEnd() throws Exception {
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    ReplicationTable.setOnline(client);
-
-    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat1 =
-        Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(true).build();
-    Status stat2 =
-        Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(true).build();
-    Status stat3 =
-        Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(true).build();
-    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", TableId.of("1"));
-    ReplicationTarget target2 = new ReplicationTarget("peer2", "table2", TableId.of("1"));
-    ReplicationTarget target3 = new ReplicationTarget("peer3", "table3", TableId.of("1"));
-
-    // Create a single work record for a file to some peer
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    Mutation m = new Mutation(file);
-    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
-    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
-    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
-    bw.addMutation(m);
-    bw.close();
-
-    updater.run();
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      s.setRange(Range.exact(file));
-      StatusSection.limit(s);
-      Entry<Key,Value> entry = getOnlyElement(s);
-
-      assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
-      assertEquals(entry.getKey().getColumnQualifier().toString(),
-          target1.getSourceTableId().canonical());
-
-      // We should only rely on the correct begin attribute being returned
-      Status actual = Status.parseFrom(entry.getValue().get());
-      assertEquals(1, actual.getBegin());
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
deleted file mode 100644
index 7bcb03c..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ /dev/null
@@ -1,431 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-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.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.DataFileColumnFamily;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationTable;
-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.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.log.WalStateManager;
-import org.apache.accumulo.server.log.WalStateManager.WalState;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * ACCUMULO-3302 series of tests which ensure that a WAL is prematurely closed when a TServer may
- * still continue to use it. Checking that no tablet references a WAL is insufficient to determine
- * if a WAL will never be used in the future.
- */
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacBase {
-  private static final Logger log =
-      LoggerFactory.getLogger(GarbageCollectorCommunicatesWithTServersIT.class);
-
-  private final int GC_PERIOD_SECONDS = 1;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(2);
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
-    cfg.setNumTservers(1);
-    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
-    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.GC_CYCLE_DELAY, GC_PERIOD_SECONDS + "s");
-    // Wait longer to try to let the replication table come online before a cycle runs
-    cfg.setProperty(Property.GC_CYCLE_START, "10s");
-    cfg.setProperty(Property.REPLICATION_NAME, "manager");
-    // Set really long delays for the manager to do stuff for replication. We don't need
-    // it to be doing anything, so just let it sleep
-    cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "240s");
-    cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "240s");
-    cfg.setProperty(Property.REPLICATION_DRIVER_DELAY, "240s");
-    // Pull down the maximum size of the wal so we can test close()'ing it.
-    cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "1M");
-    coreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  /**
-   * Fetch all of the WALs referenced by tablets in the metadata table for this table
-   */
-  private Set<String> getWalsForTable(String tableName) throws Exception {
-    final ServerContext context = getServerContext();
-    final String tableId = context.tableOperations().tableIdMap().get(tableName);
-
-    assertNotNull(tableId, "Could not determine table ID for " + tableName);
-
-    WalStateManager wals = new WalStateManager(context);
-
-    Set<String> result = new HashSet<>();
-    for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
-      log.debug("Reading WALs: {}={}", entry.getKey(), entry.getValue());
-      result.add(entry.getKey().toString());
-    }
-    return result;
-  }
-
-  /**
-   * Fetch all of the rfiles referenced by tablets in the metadata table for this table
-   */
-  private Set<String> getFilesForTable(String tableName) throws Exception {
-    final AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-    final TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
-
-    assertNotNull(tableId, "Could not determine table ID for " + tableName);
-
-    Set<String> rfiles = new HashSet<>();
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      Range r = TabletsSection.getRange(tableId);
-      s.setRange(r);
-      s.fetchColumnFamily(DataFileColumnFamily.NAME);
-
-      for (Entry<Key,Value> entry : s) {
-        log.debug("Reading RFiles: {}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-        // uri://path/to/wal
-        String cq = entry.getKey().getColumnQualifier().toString();
-        String path = new Path(cq).toString();
-        log.debug("Normalize path to rfile: {}", path);
-        rfiles.add(path);
-      }
-    }
-    return rfiles;
-  }
-
-  /**
-   * Get the replication status messages for the given table that exist in the metadata table (~repl
-   * entries)
-   */
-  private Map<String,Status> getMetadataStatusForTable(String tableName) throws Exception {
-    final AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-    final String tableId = client.tableOperations().tableIdMap().get(tableName);
-
-    assertNotNull(tableId, "Could not determine table ID for " + tableName);
-
-    Map<String,Status> fileToStatus = new HashMap<>();
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      Range r = ReplicationSection.getRange();
-      s.setRange(r);
-      s.fetchColumn(ReplicationSection.COLF, new Text(tableId));
-
-      for (Entry<Key,Value> entry : s) {
-        Text file = new Text();
-        ReplicationSection.getFile(entry.getKey(), file);
-        Status status = Status.parseFrom(entry.getValue().get());
-        log.info("Got status for {}: {}", file, ProtobufUtil.toString(status));
-        fileToStatus.put(file.toString(), status);
-      }
-    }
-    return fileToStatus;
-  }
-
-  @Test
-  public void testActiveWalPrecludesClosing() throws Exception {
-    final String table = getUniqueNames(1)[0];
-    final AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-
-    // Bring the replication table online first and foremost
-    ReplicationTable.setOnline(client);
-
-    log.info("Creating {}", table);
-    client.tableOperations().create(table);
-
-    client.tableOperations().setProperty(table, Property.TABLE_REPLICATION.getKey(), "true");
-
-    log.info("Writing a few mutations to the table");
-
-    try (BatchWriter bw = client.createBatchWriter(table)) {
-      byte[] empty = new byte[0];
-      for (int i = 0; i < 5; i++) {
-        Mutation m = new Mutation(Integer.toString(i));
-        m.put(empty, empty, empty);
-        bw.addMutation(m);
-      }
-    }
-
-    log.info(
-        "Checking that metadata only has two WALs recorded for this table (inUse, and opened)");
-
-    Set<String> wals = getWalsForTable(table);
-    assertEquals(2, wals.size(), "Expected to only find two WALs for the table");
-
-    // Flush our test table to remove the WAL references in it
-    client.tableOperations().flush(table, null, null, true);
-    // Flush the metadata table too because it will have a reference to the WAL
-    client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-
-    log.info("Waiting for replication table to come online");
-
-    log.info("Fetching replication statuses from metadata table");
-
-    Map<String,Status> fileToStatus = getMetadataStatusForTable(table);
-
-    assertEquals(1, fileToStatus.size(), "Expected to only find one replication status message");
-
-    String walName = fileToStatus.keySet().iterator().next();
-    wals.retainAll(fileToStatus.keySet());
-    assertEquals(1, wals.size());
-
-    Status status = fileToStatus.get(walName);
-
-    assertFalse(status.getClosed(), "Expected Status for file to not be closed");
-
-    Set<String> filesForTable = getFilesForTable(table);
-    assertEquals(1, filesForTable.size(), "Expected to only find one rfile for table");
-    log.info("Files for table before MajC: {}", filesForTable);
-
-    // Issue a MajC to roll a new file in HDFS
-    client.tableOperations().compact(table, null, null, false, true);
-
-    Set<String> filesForTableAfterCompaction = getFilesForTable(table);
-
-    log.info("Files for table after MajC: {}", filesForTableAfterCompaction);
-
-    assertEquals(1, filesForTableAfterCompaction.size(),
-        "Expected to only find one rfile for table");
-    assertNotEquals(filesForTableAfterCompaction, filesForTable,
-        "Expected the files before and after compaction to differ");
-
-    // Use the rfile which was just replaced by the MajC to determine when the GC has ran
-    Path fileToBeDeleted = new Path(filesForTable.iterator().next());
-    FileSystem fs = getCluster().getFileSystem();
-
-    boolean fileExists = fs.exists(fileToBeDeleted);
-    while (fileExists) {
-      log.info("File which should get deleted still exists: {}", fileToBeDeleted);
-      Thread.sleep(2000);
-      fileExists = fs.exists(fileToBeDeleted);
-    }
-
-    Map<String,Status> fileToStatusAfterMinc = getMetadataStatusForTable(table);
-    assertEquals(1, fileToStatusAfterMinc.size(),
-        "Expected to still find only one replication status message: " + fileToStatusAfterMinc);
-
-    assertEquals(fileToStatus, fileToStatusAfterMinc,
-        "Status before and after MinC should be identical");
-  }
-
-  @Test
-  public void testUnreferencedWalInTserverIsClosed() throws Exception {
-    final String[] names = getUniqueNames(2);
-    // `table` will be replicated, `otherTable` is only used to roll the WAL on the tserver
-    final String table = names[0], otherTable = names[1];
-    final AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-
-    // Bring the replication table online first and foremost
-    ReplicationTable.setOnline(client);
-
-    log.info("Creating {}", table);
-    client.tableOperations().create(table);
-
-    client.tableOperations().setProperty(table, Property.TABLE_REPLICATION.getKey(), "true");
-
-    log.info("Writing a few mutations to the table");
-
-    byte[] empty = new byte[0];
-    try (BatchWriter bw = client.createBatchWriter(table)) {
-      for (int i = 0; i < 5; i++) {
-        Mutation m = new Mutation(Integer.toString(i));
-        m.put(empty, empty, empty);
-        bw.addMutation(m);
-      }
-      log.info("Flushing mutations to the server");
-    }
-
-    log.info("Checking that metadata only has one WAL recorded for this table");
-
-    Set<String> wals = getWalsForTable(table);
-    assertEquals(2, wals.size(), "Expected to only find two WAL for the table");
-
-    log.info("Compacting the table which will remove all WALs from the tablets");
-
-    // Flush our test table to remove the WAL references in it
-    client.tableOperations().flush(table, null, null, true);
-    // Flush the metadata table too because it will have a reference to the WAL
-    client.tableOperations().flush(MetadataTable.NAME, null, null, true);
-
-    log.info("Fetching replication statuses from metadata table");
-
-    Map<String,Status> fileToStatus = getMetadataStatusForTable(table);
-
-    assertEquals(1, fileToStatus.size(), "Expected to only find one replication status message");
-
-    String walName = fileToStatus.keySet().iterator().next();
-    assertTrue(wals.contains(walName),
-        "Expected log file name from tablet to equal replication entry");
-
-    Status status = fileToStatus.get(walName);
-
-    assertFalse(status.getClosed(), "Expected Status for file to not be closed");
-
-    Set<String> filesForTable = getFilesForTable(table);
-    assertEquals(1, filesForTable.size(), "Expected to only find one rfile for table");
-    log.info("Files for table before MajC: {}", filesForTable);
-
-    // Issue a MajC to roll a new file in HDFS
-    client.tableOperations().compact(table, null, null, false, true);
-
-    Set<String> filesForTableAfterCompaction = getFilesForTable(table);
-
-    log.info("Files for table after MajC: {}", filesForTableAfterCompaction);
-
-    assertEquals(1, filesForTableAfterCompaction.size(),
-        "Expected to only find one rfile for table");
-    assertNotEquals(filesForTableAfterCompaction, filesForTable,
-        "Expected the files before and after compaction to differ");
-
-    // Use the rfile which was just replaced by the MajC to determine when the GC has ran
-    Path fileToBeDeleted = new Path(filesForTable.iterator().next());
-    FileSystem fs = getCluster().getFileSystem();
-
-    boolean fileExists = fs.exists(fileToBeDeleted);
-    while (fileExists) {
-      log.info("File which should get deleted still exists: {}", fileToBeDeleted);
-      Thread.sleep(2000);
-      fileExists = fs.exists(fileToBeDeleted);
-    }
-
-    // At this point in time, we *know* that the GarbageCollector has run which means that the
-    // Status
-    // for our WAL should not be altered.
-
-    Map<String,Status> fileToStatusAfterMinc = getMetadataStatusForTable(table);
-    assertEquals(1, fileToStatusAfterMinc.size(),
-        "Expected to still find only one replication status message: " + fileToStatusAfterMinc);
-
-    /*
-     * To verify that the WALs is still getting closed, we have to force the tserver to close the
-     * existing WAL and open a new one instead. The easiest way to do this is to write a load of
-     * data that will exceed the 1.33% full threshold that the logger keeps track of
-     */
-
-    client.tableOperations().create(otherTable);
-    try (BatchWriter bw = client.createBatchWriter(otherTable)) {
-      // 500k
-      byte[] bigValue = new byte[1024 * 500];
-      Arrays.fill(bigValue, (byte) 1);
-      // 500k * 50
-      for (int i = 0; i < 50; i++) {
-        Mutation m = new Mutation(Integer.toString(i));
-        m.put(empty, empty, bigValue);
-        bw.addMutation(m);
-        if (i % 10 == 0) {
-          bw.flush();
-        }
-      }
-    }
-
-    client.tableOperations().flush(otherTable, null, null, true);
-
-    // Get the tservers which the manager deems as active
-    final ClientContext context = (ClientContext) client;
-    List<String> tservers = ThriftClientTypes.MANAGER.execute(context,
-        mgr -> mgr.getActiveTservers(TraceUtil.traceInfo(), context.rpcCreds()));
-
-    assertEquals(1, tservers.size(), "Expected only one active tservers");
-
-    HostAndPort tserver = HostAndPort.fromString(tservers.get(0));
-
-    // Get the active WALs from that server
-    log.info("Fetching active WALs from {}", tserver);
-
-    Client cli = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, tserver, context);
-    List<String> activeWalsForTserver =
-        cli.getActiveLogs(TraceUtil.traceInfo(), context.rpcCreds());
-
-    log.info("Active wals: {}", activeWalsForTserver);
-
-    assertEquals(1, activeWalsForTserver.size(), "Expected to find only one active WAL");
-
-    String activeWal = new Path(activeWalsForTserver.get(0)).toString();
-
-    assertNotEquals("Current active WAL on tserver should not be the original WAL we saw", walName,
-        activeWal);
-
-    log.info("Ensuring that replication status does get closed after WAL is no"
-        + " longer in use by Tserver");
-
-    do {
-      Map<String,Status> replicationStatuses = getMetadataStatusForTable(table);
-
-      log.info("Got replication status messages {}", replicationStatuses);
-      assertEquals(1, replicationStatuses.size(),
-          "Did not expect to find additional status records");
-
-      status = replicationStatuses.values().iterator().next();
-      log.info("Current status: {}", ProtobufUtil.toString(status));
-
-      if (status.getClosed()) {
-        return;
-      }
-
-      log.info("Status is not yet closed, waiting for garbage collector to close it");
-
-      Thread.sleep(2000);
-    } while (true);
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
deleted file mode 100644
index 39e2cef..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
+++ /dev/null
@@ -1,276 +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.test.replication;
-
-import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.security.PrivilegedExceptionAction;
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.cluster.ClusterUser;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-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.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.harness.AccumuloITBase;
-import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
-import org.apache.accumulo.harness.MiniClusterHarness;
-import org.apache.accumulo.harness.TestingKdc;
-import org.apache.accumulo.manager.replication.SequentialWorkAssigner;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.miniclusterImpl.ProcessReference;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.test.functional.KerberosIT;
-import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Tag;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Ensure that replication occurs using keytabs instead of password (not to mention SASL)
- */
-@SuppressWarnings("deprecation")
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Tag(MINI_CLUSTER_ONLY)
-@Deprecated
-public class KerberosReplicationIT extends AccumuloITBase {
-  private static final Logger log = LoggerFactory.getLogger(KerberosIT.class);
-
-  private static TestingKdc kdc;
-  private static String krbEnabledForITs = null;
-  private static ClusterUser rootUser;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(3);
-  }
-
-  @BeforeAll
-  public static void startKdc() throws Exception {
-    kdc = new TestingKdc();
-    kdc.start();
-    krbEnabledForITs = System.getProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION);
-    if (krbEnabledForITs == null || !Boolean.parseBoolean(krbEnabledForITs)) {
-      System.setProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION, "true");
-    }
-    rootUser = kdc.getRootUser();
-  }
-
-  @AfterAll
-  public static void stopKdc() {
-    if (kdc != null) {
-      kdc.stop();
-    }
-    if (krbEnabledForITs != null) {
-      System.setProperty(MiniClusterHarness.USE_KERBEROS_FOR_IT_OPTION, krbEnabledForITs);
-    }
-  }
-
-  private MiniAccumuloClusterImpl primary, peer;
-  private String PRIMARY_NAME = "primary", PEER_NAME = "peer";
-
-  private MiniClusterConfigurationCallback getConfigCallback(final String name) {
-    return new MiniClusterConfigurationCallback() {
-      @Override
-      public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
-        cfg.setNumTservers(1);
-        cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
-        cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-        cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "2M");
-        cfg.setProperty(Property.GC_CYCLE_START, "1s");
-        cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
-        cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-        cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-        cfg.setProperty(Property.REPLICATION_NAME, name);
-        cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
-        cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
-        cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
-        coreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-        coreSite.set("fs.defaultFS", "file:///");
-      }
-    };
-  }
-
-  @BeforeEach
-  public void setup() throws Exception {
-    MiniClusterHarness harness = new MiniClusterHarness();
-
-    // Create a primary and a peer instance, both with the same "root" user
-    primary = harness.create(getClass().getName(), testName(), new PasswordToken("unused"),
-        getConfigCallback(PRIMARY_NAME), kdc);
-    primary.start();
-
-    peer = harness.create(getClass().getName(), testName() + "_peer", new PasswordToken("unused"),
-        getConfigCallback(PEER_NAME), kdc);
-    peer.start();
-
-    // Enable kerberos auth
-    Configuration conf = new Configuration(false);
-    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-  }
-
-  @AfterEach
-  public void teardown() throws Exception {
-    if (peer != null) {
-      peer.stop();
-    }
-    if (primary != null) {
-      primary.stop();
-    }
-    UserGroupInformation.setConfiguration(new Configuration(false));
-  }
-
-  @Test
-  public void dataReplicatedToCorrectTable() throws Exception {
-    // Login as the root user
-    final UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
-        rootUser.getPrincipal(), rootUser.getKeytab().toURI().toString());
-    ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
-      log.info("testing {}", ugi);
-      final KerberosToken token = new KerberosToken();
-      try (
-          AccumuloClient primaryclient =
-              primary.createAccumuloClient(rootUser.getPrincipal(), token);
-          AccumuloClient peerclient = peer.createAccumuloClient(rootUser.getPrincipal(), token)) {
-
-        ClusterUser replicationUser = kdc.getClientPrincipal(0);
-
-        // Create user for replication to the peer
-        peerclient.securityOperations().createLocalUser(replicationUser.getPrincipal(), null);
-
-        primaryclient.instanceOperations().setProperty(
-            Property.REPLICATION_PEER_USER.getKey() + PEER_NAME, replicationUser.getPrincipal());
-        primaryclient.instanceOperations().setProperty(
-            Property.REPLICATION_PEER_KEYTAB.getKey() + PEER_NAME,
-            replicationUser.getKeytab().getAbsolutePath());
-
-        // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-        ClientInfo info = ClientInfo.from(peerclient.properties());
-        primaryclient.instanceOperations().setProperty(
-            Property.REPLICATION_PEERS.getKey() + PEER_NAME,
-            ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-                AccumuloReplicaSystem.buildConfiguration(info.getInstanceName(),
-                    info.getZooKeepers())));
-
-        String primaryTable1 = "primary", peerTable1 = "peer";
-
-        // Create tables
-        peerclient.tableOperations().create(peerTable1);
-        String peerTableId1 = peerclient.tableOperations().tableIdMap().get(peerTable1);
-        assertNotNull(peerTableId1);
-
-        Map<String,String> props = new HashMap<>();
-        props.put(Property.TABLE_REPLICATION.getKey(), "true");
-        // Replicate this table to the peerClusterName in a table with the peerTableId table id
-        props.put(Property.TABLE_REPLICATION_TARGET.getKey() + PEER_NAME, peerTableId1);
-
-        primaryclient.tableOperations().create(primaryTable1,
-            new NewTableConfiguration().setProperties(props));
-        String managerTableId1 = primaryclient.tableOperations().tableIdMap().get(primaryTable1);
-        assertNotNull(managerTableId1);
-
-        // Grant write permission
-        peerclient.securityOperations().grantTablePermission(replicationUser.getPrincipal(),
-            peerTable1, TablePermission.WRITE);
-
-        // Write some data to table1
-        long managerTable1Records = 0L;
-        try (BatchWriter bw = primaryclient.createBatchWriter(primaryTable1)) {
-          for (int rows = 0; rows < 2500; rows++) {
-            Mutation m = new Mutation(primaryTable1 + rows);
-            for (int cols = 0; cols < 100; cols++) {
-              String value = Integer.toString(cols);
-              m.put(value, "", value);
-              managerTable1Records++;
-            }
-            bw.addMutation(m);
-          }
-        }
-
-        log.info("Wrote all data to primary cluster");
-
-        Set<String> filesFor1 =
-            primaryclient.replicationOperations().referencedFiles(primaryTable1);
-
-        // Restart the tserver to force a close on the WAL
-        for (ProcessReference proc : primary.getProcesses().get(ServerType.TABLET_SERVER)) {
-          primary.killProcess(ServerType.TABLET_SERVER, proc);
-        }
-        primary.exec(TabletServer.class);
-
-        log.info("Restarted the tserver");
-
-        // Read the data -- the tserver is back up and running and tablets are assigned
-        try (Scanner scanner = primaryclient.createScanner(primaryTable1, Authorizations.EMPTY)) {
-          scanner.forEach((k, v) -> {});
-        }
-
-        // Wait for both tables to be replicated
-        log.info("Waiting for {} for {}", filesFor1, primaryTable1);
-        primaryclient.replicationOperations().drain(primaryTable1, filesFor1);
-
-        long countTable = 0L;
-        try (var scanner = peerclient.createScanner(peerTable1, Authorizations.EMPTY)) {
-          for (Entry<Key,Value> entry : scanner) {
-            countTable++;
-            assertTrue(entry.getKey().getRow().toString().startsWith(primaryTable1),
-                "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                    + entry.getValue());
-          }
-        }
-
-        log.info("Found {} records in {}", countTable, peerTable1);
-        assertEquals(managerTable1Records, countTable);
-
-        return null;
-      }
-    });
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java b/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java
deleted file mode 100644
index 3727e5b..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java
+++ /dev/null
@@ -1,103 +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.test.replication;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.ReplicaSystem;
-import org.apache.accumulo.server.replication.ReplicaSystemHelper;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Fake ReplicaSystem which returns that the data was fully replicated after some sleep period (in
- * milliseconds)
- * <p>
- * Default sleep amount is 0ms
- */
-@Deprecated
-public class MockReplicaSystem implements ReplicaSystem {
-  private static final Logger log = LoggerFactory.getLogger(MockReplicaSystem.class);
-
-  private long sleep = 0;
-
-  @Override
-  public Status replicate(Path p, Status status, ReplicationTarget target,
-      ReplicaSystemHelper helper) {
-    Status newStatus;
-    if (status.getClosed() && status.getInfiniteEnd()) {
-      Status.Builder builder = Status.newBuilder(status);
-      if (status.getInfiniteEnd()) {
-        builder.setBegin(Long.MAX_VALUE);
-      } else {
-        builder.setBegin(status.getEnd());
-      }
-      newStatus = builder.build();
-    } else {
-      log.info("{} with status {} is not closed and with infinite length, ignoring", p, status);
-      newStatus = status;
-    }
-
-    log.debug("Sleeping for {}ms before finishing replication on {}", sleep, p);
-    try {
-      Thread.sleep(sleep);
-    } catch (InterruptedException e) {
-      log.error("Interrupted while sleeping, will report no progress", e);
-      Thread.currentThread().interrupt();
-      return status;
-    }
-
-    log.info("For {}, received {}, returned {}", p, ProtobufUtil.toString(status),
-        ProtobufUtil.toString(newStatus));
-    try {
-      helper.recordNewStatus(p, newStatus, target);
-    } catch (TableNotFoundException e) {
-      log.error(
-          "Tried to update status in replication table for {} as {}, but the table did not exist",
-          p, ProtobufUtil.toString(newStatus), e);
-      return status;
-    } catch (AccumuloException e) {
-      log.error("Tried to record new status in replication table for {} as {}, but got an error", p,
-          ProtobufUtil.toString(newStatus), e);
-      return status;
-    }
-
-    return newStatus;
-  }
-
-  @Override
-  public void configure(ServerContext context, String configuration) {
-    if (configuration.isBlank()) {
-      log.debug("No configuration, using default sleep of {}", sleep);
-      return;
-    }
-
-    try {
-      sleep = Long.parseLong(configuration);
-    } catch (NumberFormatException e) {
-      log.warn("Could not parse {} as an integer, using default sleep of {}", configuration, sleep,
-          e);
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
deleted file mode 100644
index 5e85d9b..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ /dev/null
@@ -1,814 +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.test.replication;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.PartialKey;
-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.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.SequentialWorkAssigner;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.miniclusterImpl.ProcessReference;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Replication tests which start at least two MAC instances and replicate data between them
- */
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class MultiInstanceReplicationIT extends ConfigurableMacBase {
-  private static final Logger log = LoggerFactory.getLogger(MultiInstanceReplicationIT.class);
-
-  private ExecutorService executor;
-
-  @BeforeEach
-  public void createExecutor() {
-    executor = Executors.newSingleThreadExecutor();
-  }
-
-  @AfterEach
-  public void stopExecutor() {
-    if (executor != null) {
-      executor.shutdownNow();
-    }
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
-    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
-    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "2M");
-    cfg.setProperty(Property.GC_CYCLE_START, "1s");
-    cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
-    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-    cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-    cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
-    cfg.setProperty(Property.REPLICATION_NAME, "manager");
-    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
-    cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
-    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  /**
-   * Use the same SSL and credential provider configuration that is set up by AbstractMacIT for the
-   * other MAC used for replication
-   */
-  private void updatePeerConfigFromPrimary(MiniAccumuloConfigImpl primaryCfg,
-      MiniAccumuloConfigImpl peerCfg) {
-    // Set the same SSL information from the primary when present
-    Map<String,String> primarySiteConfig = primaryCfg.getSiteConfig();
-    if ("true".equals(primarySiteConfig.get(Property.INSTANCE_RPC_SSL_ENABLED.getKey()))) {
-      Map<String,String> peerSiteConfig = new HashMap<>();
-      peerSiteConfig.put(Property.INSTANCE_RPC_SSL_ENABLED.getKey(), "true");
-      String keystorePath = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey());
-      assertNotNull(keystorePath, "Keystore Path was null");
-      peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PATH.getKey(), keystorePath);
-      String truststorePath = primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey());
-      assertNotNull(truststorePath, "Truststore Path was null");
-      peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PATH.getKey(), truststorePath);
-
-      // Passwords might be stored in CredentialProvider
-      String keystorePassword = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey());
-      if (keystorePassword != null) {
-        peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey(), keystorePassword);
-      }
-      String truststorePassword =
-          primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey());
-      if (truststorePassword != null) {
-        peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
-      }
-
-      System.out.println("Setting site configuration for peer " + peerSiteConfig);
-      peerCfg.setSiteConfig(peerSiteConfig);
-    }
-
-    // Use the CredentialProvider if the primary also uses one
-    String credProvider =
-        primarySiteConfig.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
-    if (credProvider != null) {
-      Map<String,String> peerSiteConfig = peerCfg.getSiteConfig();
-      peerSiteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(),
-          credProvider);
-      peerCfg.setSiteConfig(peerSiteConfig);
-    }
-  }
-
-  @Test
-  public void dataWasReplicatedToThePeer() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-
-    MiniAccumuloClusterImpl peerCluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peerCluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      ReplicationTable.setOnline(clientManager);
-
-      String peerUserName = "peer", peerPassword = "foo";
-
-      String peerClusterName = "peer";
-
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
-                  peerCluster.getZooKeepers())));
-
-      final String managerTable = "manager", peerTable = "peer";
-
-      clientPeer.tableOperations().create(peerTable, new NewTableConfiguration());
-      String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
-      assertNotNull(peerTableId);
-
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable,
-          TablePermission.WRITE);
-
-      // Replicate this table to the peerClusterName in a table with the peerTableId table id
-      Map<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
-
-      clientManager.tableOperations().create(managerTable,
-          new NewTableConfiguration().setProperties(props));
-      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
-      assertNotNull(managerTableId);
-
-      // Write some data to table1
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
-        for (int rows = 0; rows < 5000; rows++) {
-          Mutation m = new Mutation(Integer.toString(rows));
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      final Set<String> filesNeedingReplication =
-          clientManager.replicationOperations().referencedFiles(managerTable);
-
-      log.info("Files to replicate: " + filesNeedingReplication);
-
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-      cluster.exec(TabletServer.class);
-
-      log.info("TabletServer restarted");
-      try (Scanner scanner = ReplicationTable.getScanner(clientManager)) {
-        scanner.forEach((k, v) -> {});
-      }
-      log.info("TabletServer is online");
-
-      while (!ReplicationTable.isOnline(clientManager)) {
-        log.info("Replication table still offline, waiting");
-        Thread.sleep(5000);
-      }
-
-      log.info("");
-      log.info("Fetching metadata records:");
-      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> kv : scanner) {
-          if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-          } else {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(), kv.getValue());
-          }
-        }
-      }
-
-      log.info("");
-      log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientManager)) {
-        for (Entry<Key,Value> kv : scanner) {
-          log.info("{} {}", kv.getKey().toStringNoTruncate(),
-              ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-        }
-      }
-
-      Future<Boolean> future = executor.submit(() -> {
-        long then = System.currentTimeMillis();
-        clientManager.replicationOperations().drain(managerTable, filesNeedingReplication);
-        long now = System.currentTimeMillis();
-        log.info("Drain completed in " + (now - then) + "ms");
-        return true;
-      });
-
-      try {
-        future.get(60, TimeUnit.SECONDS);
-      } catch (TimeoutException e) {
-        future.cancel(true);
-        fail("Drain did not finish within 60 seconds");
-      } finally {
-        executor.shutdownNow();
-      }
-
-      log.info("drain completed");
-
-      log.info("");
-      log.info("Fetching metadata records:");
-      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> kv : scanner) {
-          if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-          } else {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(), kv.getValue());
-          }
-        }
-      }
-
-      log.info("");
-      log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientManager)) {
-        for (Entry<Key,Value> kv : scanner) {
-          log.info("{} {}", kv.getKey().toStringNoTruncate(),
-              ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-        }
-      }
-
-      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
-          Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
-        Entry<Key,Value> managerEntry = null, peerEntry = null;
-        while (managerIter.hasNext() && peerIter.hasNext()) {
-          managerEntry = managerIter.next();
-          peerEntry = peerIter.next();
-          assertEquals(0,
-              managerEntry.getKey().compareTo(peerEntry.getKey(),
-                  PartialKey.ROW_COLFAM_COLQUAL_COLVIS),
-              managerEntry.getKey() + " was not equal to " + peerEntry.getKey());
-          assertEquals(managerEntry.getValue(), peerEntry.getValue());
-        }
-
-        log.info("Last manager entry: {}", managerEntry);
-        log.info("Last peer entry: {}", peerEntry);
-
-        assertFalse(managerIter.hasNext(), "Had more data to read from the manager");
-        assertFalse(peerIter.hasNext(), "Had more data to read from the peer");
-      }
-    } finally {
-      peerCluster.stop();
-    }
-  }
-
-  @Test
-  public void dataReplicatedToCorrectTable() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-
-    MiniAccumuloClusterImpl peer1Cluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peer1Cluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-      String peerClusterName = "peer";
-      String peerUserName = "peer", peerPassword = "foo";
-
-      // Create local user
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
-                  peer1Cluster.getZooKeepers())));
-
-      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
-          peerTable2 = "peer2";
-
-      // Create tables
-      clientPeer.tableOperations().create(peerTable1, new NewTableConfiguration());
-      String peerTableId1 = clientPeer.tableOperations().tableIdMap().get(peerTable1);
-      assertNotNull(peerTableId1);
-
-      clientPeer.tableOperations().create(peerTable2, new NewTableConfiguration());
-      String peerTableId2 = clientPeer.tableOperations().tableIdMap().get(peerTable2);
-      assertNotNull(peerTableId2);
-
-      Map<String,String> props1 = new HashMap<>();
-      props1.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
-
-      clientManager.tableOperations().create(managerTable1,
-          new NewTableConfiguration().setProperties(props1));
-      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
-      assertNotNull(managerTableId1);
-      Map<String,String> props2 = new HashMap<>();
-      props2.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
-
-      clientManager.tableOperations().create(managerTable2,
-          new NewTableConfiguration().setProperties(props2));
-      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
-      assertNotNull(managerTableId2);
-
-      // Give our replication user the ability to write to the tables
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
-          TablePermission.WRITE);
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable2,
-          TablePermission.WRITE);
-
-      // Write some data to table1
-      long managerTable1Records = 0L;
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable1 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-            managerTable1Records++;
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      // Write some data to table2
-      long managerTable2Records = 0L;
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable2 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-            managerTable2Records++;
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      Set<String> filesFor1 = clientManager.replicationOperations().referencedFiles(managerTable1),
-          filesFor2 = clientManager.replicationOperations().referencedFiles(managerTable2);
-
-      log.info("Files to replicate for table1: " + filesFor1);
-      log.info("Files to replicate for table2: " + filesFor2);
-
-      // Restart the tserver to force a close on the WAL
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-      cluster.exec(TabletServer.class);
-
-      log.info("Restarted the tserver");
-
-      // Read the data -- the tserver is back up and running
-      try (Scanner scanner = clientManager.createScanner(managerTable1, Authorizations.EMPTY)) {
-        scanner.forEach((k, v) -> {});
-      }
-
-      while (!ReplicationTable.isOnline(clientManager)) {
-        log.info("Replication table still offline, waiting");
-        Thread.sleep(5000);
-      }
-
-      // Wait for both tables to be replicated
-      log.info("Waiting for {} for {}", filesFor1, managerTable1);
-      clientManager.replicationOperations().drain(managerTable1, filesFor1);
-
-      log.info("Waiting for {} for {}", filesFor2, managerTable2);
-      clientManager.replicationOperations().drain(managerTable2, filesFor2);
-
-      long countTable = 0L;
-      try (var scanner = clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> entry : scanner) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable1),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-      }
-
-      log.info("Found {} records in {}", countTable, peerTable1);
-      assertEquals(managerTable1Records, countTable);
-
-      countTable = 0L;
-      try (var scanner = clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> entry : scanner) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable2),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-      }
-
-      log.info("Found {} records in {}", countTable, peerTable2);
-      assertEquals(managerTable2Records, countTable);
-
-    } finally {
-      peer1Cluster.stop();
-    }
-  }
-
-  @Test
-  public void dataWasReplicatedToThePeerWithoutDrain() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-
-    MiniAccumuloClusterImpl peerCluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peerCluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      String peerUserName = "repl";
-      String peerPassword = "passwd";
-
-      // Create a user on the peer for replication to use
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      String peerClusterName = "peer";
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
-                  peerCluster.getZooKeepers())));
-
-      // Configure the credentials we should use to authenticate ourselves to the peer for
-      // replication
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      String managerTable = "manager", peerTable = "peer";
-      clientPeer.tableOperations().create(peerTable, new NewTableConfiguration());
-      String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
-      assertNotNull(peerTableId);
-
-      // Give our replication user the ability to write to the table
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable,
-          TablePermission.WRITE);
-
-      Map<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      // Replicate this table to the peerClusterName in a table with the peerTableId table id
-      props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
-      clientManager.tableOperations().create(managerTable,
-          new NewTableConfiguration().setProperties(props));
-      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
-      assertNotNull(managerTableId);
-
-      // Write some data to table1
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
-        for (int rows = 0; rows < 5000; rows++) {
-          Mutation m = new Mutation(Integer.toString(rows));
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      Set<String> files = clientManager.replicationOperations().referencedFiles(managerTable);
-
-      log.info("Files to replicate:" + files);
-
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-
-      cluster.exec(TabletServer.class);
-
-      while (!ReplicationTable.isOnline(clientManager)) {
-        log.info("Replication table still offline, waiting");
-        Thread.sleep(5000);
-      }
-
-      try (Scanner scanner = clientManager.createScanner(managerTable, Authorizations.EMPTY)) {
-        scanner.forEach((k, v) -> {});
-      }
-
-      try (var scanner = ReplicationTable.getScanner(clientManager)) {
-        for (Entry<Key,Value> kv : scanner) {
-          log.debug("{} {}", kv.getKey().toStringNoTruncate(),
-              ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-        }
-      }
-
-      clientManager.replicationOperations().drain(managerTable, files);
-
-      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
-          Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
-        while (managerIter.hasNext() && peerIter.hasNext()) {
-          Entry<Key,Value> managerEntry = managerIter.next(), peerEntry = peerIter.next();
-          assertEquals(0,
-              managerEntry.getKey().compareTo(peerEntry.getKey(),
-                  PartialKey.ROW_COLFAM_COLQUAL_COLVIS),
-              peerEntry.getKey() + " was not equal to " + peerEntry.getKey());
-          assertEquals(managerEntry.getValue(), peerEntry.getValue());
-        }
-
-        assertFalse(managerIter.hasNext(), "Had more data to read from the manager");
-        assertFalse(peerIter.hasNext(), "Had more data to read from the peer");
-      }
-    } finally {
-      peerCluster.stop();
-    }
-  }
-
-  @Test
-  public void dataReplicatedToCorrectTableWithoutDrain() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-
-    MiniAccumuloClusterImpl peer1Cluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peer1Cluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      String peerClusterName = "peer";
-
-      String peerUserName = "repl";
-      String peerPassword = "passwd";
-
-      // Create a user on the peer for replication to use
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      // Configure the credentials we should use to authenticate ourselves to the peer for
-      // replication
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
-                  peer1Cluster.getZooKeepers())));
-
-      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
-          peerTable2 = "peer2";
-      // Create tables
-      clientPeer.tableOperations().create(peerTable1, new NewTableConfiguration());
-      String peerTableId1 = clientPeer.tableOperations().tableIdMap().get(peerTable1);
-      assertNotNull(peerTableId1);
-
-      clientPeer.tableOperations().create(peerTable2, new NewTableConfiguration());
-      String peerTableId2 = clientPeer.tableOperations().tableIdMap().get(peerTable2);
-      assertNotNull(peerTableId2);
-
-      Map<String,String> props1 = new HashMap<>();
-      props1.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
-
-      clientManager.tableOperations().create(managerTable1,
-          new NewTableConfiguration().setProperties(props1));
-      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
-      assertNotNull(managerTableId1);
-
-      Map<String,String> props2 = new HashMap<>();
-      props2.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
-
-      clientManager.tableOperations().create(managerTable2,
-          new NewTableConfiguration().setProperties(props2));
-      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
-      assertNotNull(managerTableId2);
-
-      // Give our replication user the ability to write to the tables
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
-          TablePermission.WRITE);
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable2,
-          TablePermission.WRITE);
-
-      // Replicate this table to the peerClusterName in a table with the peerTableId table id
-      clientManager.tableOperations().setProperty(managerTable1,
-          Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
-      clientManager.tableOperations().setProperty(managerTable2,
-          Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
-
-      // Write some data to table1
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable1 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      // Write some data to table2
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable2 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-
-      cluster.exec(TabletServer.class);
-
-      while (!ReplicationTable.isOnline(clientManager)) {
-        log.info("Replication table still offline, waiting");
-        Thread.sleep(5000);
-      }
-
-      // Wait until we fully replicated something
-      boolean fullyReplicated = false;
-      for (int i = 0; i < 10 && !fullyReplicated; i++) {
-        sleepUninterruptibly(2, TimeUnit.SECONDS);
-
-        try (Scanner s = ReplicationTable.getScanner(clientManager)) {
-          WorkSection.limit(s);
-          for (Entry<Key,Value> entry : s) {
-            Status status = Status.parseFrom(entry.getValue().get());
-            if (StatusUtil.isFullyReplicated(status)) {
-              fullyReplicated |= true;
-            }
-          }
-        }
-      }
-
-      assertNotEquals(0, fullyReplicated);
-
-      // We have to wait for the manager to assign the replication work, a local tserver to process
-      // it, and then the remote tserver to replay it
-      // Be cautious in how quickly we assert that the data is present on the peer
-      long countTable = 0L;
-      for (int i = 0; i < 10; i++) {
-        for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable1),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-
-        log.info("Found {} records in {}", countTable, peerTable1);
-
-        if (countTable == 0L) {
-          Thread.sleep(5000);
-        } else {
-          break;
-        }
-      }
-
-      assertTrue(countTable > 0, "Found no records in " + peerTable1 + " in the peer cluster");
-
-      // We have to wait for the manager to assign the replication work, a local tserver to process
-      // it, and then the remote tserver to replay it
-      // Be cautious in how quickly we assert that the data is present on the peer
-      for (int i = 0; i < 10; i++) {
-        countTable = 0L;
-        for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable2),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-
-        log.info("Found {} records in {}", countTable, peerTable2);
-
-        if (countTable == 0L) {
-          Thread.sleep(5000);
-        } else {
-          break;
-        }
-      }
-
-      assertTrue(countTable > 0, "Found no records in " + peerTable2 + " in the peer cluster");
-
-    } finally {
-      peer1Cluster.stop();
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
deleted file mode 100644
index b27f5fd..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
+++ /dev/null
@@ -1,131 +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.test.replication;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.fate.zookeeper.ZooReader;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.HostAndPort;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.conf.Configuration;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Iterables;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class MultiTserverReplicationIT extends ConfigurableMacBase {
-  private static final Logger log = LoggerFactory.getLogger(MultiTserverReplicationIT.class);
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    // set the name to kick off the replication services
-    cfg.setProperty(Property.REPLICATION_NAME.getKey(), "test");
-    cfg.setNumTservers(2);
-  }
-
-  @Test
-  public void tserverReplicationServicePortsAreAdvertised() throws Exception {
-    // Wait for the cluster to be up
-    AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-    ClientContext context = (ClientContext) client;
-
-    // Wait for a tserver to come up to fulfill this request
-    client.tableOperations().create("foo");
-    try (Scanner s = client.createScanner("foo", Authorizations.EMPTY)) {
-      assertEquals(0, Iterables.size(s));
-
-      ZooReader zreader = context.getZooReader();
-      Set<String> tserverHost = new HashSet<>();
-      tserverHost.addAll(zreader.getChildren(
-          ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZTSERVERS));
-
-      Set<HostAndPort> replicationServices = new HashSet<>();
-
-      for (String tserver : tserverHost) {
-        try {
-          byte[] portData =
-              zreader.getData(ZooUtil.getRoot(client.instanceOperations().getInstanceId())
-                  + ReplicationConstants.ZOO_TSERVERS + "/" + tserver);
-          HostAndPort replAddress = HostAndPort.fromString(new String(portData, UTF_8));
-          replicationServices.add(replAddress);
-        } catch (Exception e) {
-          log.error("Could not find port for {}", tserver, e);
-          fail("Did not find replication port advertisement for " + tserver);
-        }
-      }
-
-      // Each tserver should also have equal replication services running internally
-      assertEquals(tserverHost.size(), replicationServices.size(),
-          "Expected an equal number of replication servicers and tservers");
-    }
-  }
-
-  @Test
-  public void managerReplicationServicePortsAreAdvertised() throws Exception {
-    // Wait for the cluster to be up
-    AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-    ClientContext context = (ClientContext) client;
-
-    // Wait for a tserver to come up to fulfill this request
-    client.tableOperations().create("foo");
-    try (Scanner s = client.createScanner("foo", Authorizations.EMPTY)) {
-      assertEquals(0, Iterables.size(s));
-
-      ZooReader zreader = context.getZooReader();
-
-      // Should have one manager instance
-      assertEquals(1, context.getManagerLocations().size());
-
-      // Get the manager thrift service addr
-      String managerAddr = getOnlyElement(context.getManagerLocations());
-
-      // Get the manager replication coordinator addr
-      String replCoordAddr =
-          new String(zreader.getData(ZooUtil.getRoot(client.instanceOperations().getInstanceId())
-              + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR), UTF_8);
-
-      // They shouldn't be the same
-      assertNotEquals(managerAddr, replCoordAddr);
-
-      // Neither should be zero as the port
-      assertNotEquals(0, HostAndPort.fromString(managerAddr).getPort());
-      assertNotEquals(0, HostAndPort.fromString(replCoordAddr).getPort());
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java b/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java
deleted file mode 100644
index 0da6271..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java
+++ /dev/null
@@ -1,364 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.IteratorSetting;
-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.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.RemoveCompleteReplicationRecords;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Iterables;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class RemoveCompleteReplicationRecordsIT extends ConfigurableMacBase {
-
-  private MockRemoveCompleteReplicationRecords rcrr;
-  private AccumuloClient client;
-
-  private static class MockRemoveCompleteReplicationRecords
-      extends RemoveCompleteReplicationRecords {
-
-    public MockRemoveCompleteReplicationRecords(AccumuloClient client) {
-      super(client);
-    }
-
-    @Override
-    public long removeCompleteRecords(AccumuloClient client, BatchScanner bs, BatchWriter bw) {
-      return super.removeCompleteRecords(client, bs, bw);
-    }
-
-  }
-
-  @BeforeEach
-  public void initialize() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    rcrr = new MockRemoveCompleteReplicationRecords(client);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    ReplicationTable.setOnline(client);
-  }
-
-  private TableId createTableId(int i) {
-    return TableId.of(Integer.toString(i));
-  }
-
-  @Test
-  public void notYetReplicationRecordsIgnored() throws Exception {
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    int numRecords = 3;
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, createTableId(i), StatusUtil.openWithUnknownLengthValue());
-      bw.addMutation(m);
-    }
-
-    bw.close();
-
-    assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-
-    try (BatchScanner bs = ReplicationTable.getBatchScanner(client, 1)) {
-      bs.setRanges(Collections.singleton(new Range()));
-      IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-      bs.addScanIterator(cfg);
-      bw = EasyMock.createMock(BatchWriter.class);
-
-      EasyMock.replay(bw);
-
-      rcrr.removeCompleteRecords(client, bs, bw);
-      assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-    }
-  }
-
-  @Test
-  public void partiallyReplicatedRecordsIgnored() throws Exception {
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    int numRecords = 3;
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, createTableId(i),
-          ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
-      bw.addMutation(m);
-    }
-
-    bw.close();
-
-    assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-
-    try (BatchScanner bs = ReplicationTable.getBatchScanner(client, 1)) {
-      bs.setRanges(Collections.singleton(new Range()));
-      IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-      bs.addScanIterator(cfg);
-      bw = EasyMock.createMock(BatchWriter.class);
-
-      EasyMock.replay(bw);
-
-      // We don't remove any records, so we can just pass in a fake BW for both
-      rcrr.removeCompleteRecords(client, bs, bw);
-      assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-    }
-  }
-
-  @Test
-  public void replicatedClosedWorkRecordsAreNotRemovedWithoutClosedStatusRecords()
-      throws Exception {
-    BatchWriter replBw = ReplicationTable.getBatchWriter(client);
-    int numRecords = 3;
-
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-
-    // Write out numRecords entries to both replication and metadata tables, none of which are fully
-    // replicated
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, createTableId(i),
-          ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
-      replBw.addMutation(m);
-    }
-
-    // Add two records that we can delete
-    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    Mutation m = new Mutation(fileToRemove);
-    StatusSection.add(m, TableId.of("5"),
-        ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(false).build()));
-    replBw.addMutation(m);
-
-    numRecords++;
-
-    fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    m = new Mutation(fileToRemove);
-    StatusSection.add(m, TableId.of("6"),
-        ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(false).build()));
-    replBw.addMutation(m);
-
-    numRecords++;
-
-    replBw.flush();
-
-    // Make sure that we have the expected number of records in both tables
-    assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-
-    // We should not remove any records because they're missing closed status
-    try (BatchScanner bs = ReplicationTable.getBatchScanner(client, 1)) {
-      bs.setRanges(Collections.singleton(new Range()));
-      IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-      bs.addScanIterator(cfg);
-
-      try {
-        assertEquals(0L, rcrr.removeCompleteRecords(client, bs, replBw));
-      } finally {
-        replBw.close();
-      }
-    }
-  }
-
-  @Test
-  public void replicatedClosedRowsAreRemoved() throws Exception {
-    BatchWriter replBw = ReplicationTable.getBatchWriter(client);
-    int numRecords = 3;
-
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-
-    long time = System.currentTimeMillis();
-    // Write out numRecords entries to both replication and metadata tables, none of which are fully
-    // replicated
-    for (int i = 0; i < numRecords; i++) {
-      builder.setCreatedTime(time++);
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      Value v = ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build());
-      StatusSection.add(m, createTableId(i), v);
-      replBw.addMutation(m);
-      m = OrderSection.createMutation(file, time);
-      OrderSection.add(m, createTableId(i), v);
-      replBw.addMutation(m);
-    }
-
-    Set<String> filesToRemove = new HashSet<>();
-    // We created two mutations for each file
-    numRecords *= 2;
-    int finalNumRecords = numRecords;
-
-    // Add two records that we can delete
-    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    filesToRemove.add(fileToRemove);
-    Mutation m = new Mutation(fileToRemove);
-    ReplicationTarget target = new ReplicationTarget("peer1", "5", TableId.of("5"));
-    Value value = ProtobufUtil.toValue(
-        builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
-    StatusSection.add(m, TableId.of("5"), value);
-    WorkSection.add(m, target.toText(), value);
-    replBw.addMutation(m);
-
-    m = OrderSection.createMutation(fileToRemove, time);
-    OrderSection.add(m, TableId.of("5"), value);
-    replBw.addMutation(m);
-    time++;
-
-    numRecords += 3;
-
-    fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    filesToRemove.add(fileToRemove);
-    m = new Mutation(fileToRemove);
-    value = ProtobufUtil.toValue(
-        builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
-    target = new ReplicationTarget("peer1", "6", TableId.of("6"));
-    StatusSection.add(m, TableId.of("6"), value);
-    WorkSection.add(m, target.toText(), value);
-    replBw.addMutation(m);
-
-    m = OrderSection.createMutation(fileToRemove, time);
-    OrderSection.add(m, TableId.of("6"), value);
-    replBw.addMutation(m);
-    time++;
-
-    numRecords += 3;
-
-    replBw.flush();
-
-    // Make sure that we have the expected number of records in both tables
-    assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-
-    // We should remove the two fully completed records we inserted
-    try (BatchScanner bs = ReplicationTable.getBatchScanner(client, 1)) {
-      bs.setRanges(Collections.singleton(new Range()));
-      StatusSection.limit(bs);
-      WorkSection.limit(bs);
-      IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-      bs.addScanIterator(cfg);
-
-      try {
-        assertEquals(4L, rcrr.removeCompleteRecords(client, bs, replBw));
-      } finally {
-        replBw.close();
-      }
-
-      int actualRecords = 0;
-      try (var scanner = ReplicationTable.getScanner(client)) {
-        for (Entry<Key,Value> entry : scanner) {
-          assertFalse(filesToRemove.contains(entry.getKey().getRow().toString()));
-          actualRecords++;
-        }
-      }
-
-      assertEquals(finalNumRecords, actualRecords);
-    }
-  }
-
-  @Test
-  public void partiallyReplicatedEntriesPrecludeRowDeletion() throws Exception {
-    BatchWriter replBw = ReplicationTable.getBatchWriter(client);
-    int numRecords = 3;
-
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-
-    // Write out numRecords entries to both replication and metadata tables, none of which are fully
-    // replicated
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, createTableId(i),
-          ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
-      replBw.addMutation(m);
-    }
-
-    // Add two records that we can delete
-    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    Mutation m = new Mutation(fileToRemove);
-    ReplicationTarget target = new ReplicationTarget("peer1", "5", TableId.of("5"));
-    Value value =
-        ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).build());
-    StatusSection.add(m, TableId.of("5"), value);
-    WorkSection.add(m, target.toText(), value);
-    target = new ReplicationTarget("peer2", "5", TableId.of("5"));
-    WorkSection.add(m, target.toText(), value);
-    target = new ReplicationTarget("peer3", "5", TableId.of("5"));
-    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(builder.setClosed(false).build()));
-    replBw.addMutation(m);
-
-    numRecords += 4;
-
-    replBw.flush();
-
-    // Make sure that we have the expected number of records in both tables
-    assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(client)));
-
-    // We should remove the two fully completed records we inserted
-    try (BatchScanner bs = ReplicationTable.getBatchScanner(client, 1)) {
-      bs.setRanges(Collections.singleton(new Range()));
-      IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-      bs.addScanIterator(cfg);
-
-      try {
-        assertEquals(0L, rcrr.removeCompleteRecords(client, bs, replBw));
-      } finally {
-        replBw.close();
-      }
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
deleted file mode 100644
index bb12bea..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ /dev/null
@@ -1,1514 +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.test.replication;
-
-import static java.util.Collections.singletonMap;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Accumulo;
-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.IteratorSetting;
-import org.apache.accumulo.core.client.IteratorSetting.Column;
-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.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-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.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.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iteratorsImpl.conf.ColumnSet;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.TServerInstance;
-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.CurrentLocationColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.gc.SimpleGarbageCollector;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.log.WalStateManager;
-import org.apache.accumulo.server.log.WalStateManager.WalState;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.server.replication.StatusCombiner;
-import org.apache.accumulo.server.replication.StatusFormatter;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
-import com.google.protobuf.TextFormat;
-
-/**
- * Replication tests which verify expected functionality using a single MAC instance. A
- * MockReplicaSystem is used to "fake" the peer instance that we're replicating to. This lets us
- * test replication in a functional way without having to worry about two real systems.
- */
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class ReplicationIT extends ConfigurableMacBase {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationIT.class);
-  private static final long MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS = 5000L;
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    // Run the manager replication loop run frequently
-    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
-    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-    cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "1M");
-    cfg.setProperty(Property.GC_CYCLE_START, "1s");
-    cfg.setProperty(Property.GC_CYCLE_DELAY, "0");
-    cfg.setProperty(Property.REPLICATION_NAME, "manager");
-    cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "1s");
-    cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_PERIOD, "1s");
-    cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
-    cfg.setNumTservers(1);
-    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  private Multimap<String,TableId> getLogs(AccumuloClient client, ServerContext context)
-      throws Exception {
-    // Map of server to tableId
-    Multimap<TServerInstance,TableId> serverToTableID = HashMultimap.create();
-    try (Scanner scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      scanner.setRange(TabletsSection.getRange());
-      scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
-      for (Entry<Key,Value> entry : scanner) {
-        var tServer = new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier());
-        TableId tableId = KeyExtent.fromMetaRow(entry.getKey().getRow()).tableId();
-        serverToTableID.put(tServer, tableId);
-      }
-      // Map of logs to tableId
-      Multimap<String,TableId> logs = HashMultimap.create();
-      WalStateManager wals = new WalStateManager(context);
-      for (Entry<TServerInstance,List<UUID>> entry : wals.getAllMarkers().entrySet()) {
-        for (UUID id : entry.getValue()) {
-          Pair<WalState,Path> state = wals.state(entry.getKey(), id);
-          for (TableId tableId : serverToTableID.get(entry.getKey())) {
-            logs.put(state.getSecond().toString(), tableId);
-          }
-        }
-      }
-      return logs;
-    }
-  }
-
-  private Multimap<String,TableId> getAllLogs(AccumuloClient client, ServerContext context)
-      throws Exception {
-    Multimap<String,TableId> logs = getLogs(client, context);
-    try (Scanner scanner = context.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
-      StatusSection.limit(scanner);
-      Text buff = new Text();
-      for (Entry<Key,Value> entry : scanner) {
-        if (Thread.interrupted()) {
-          Thread.currentThread().interrupt();
-          return logs;
-        }
-
-        StatusSection.getFile(entry.getKey(), buff);
-        String file = buff.toString();
-        TableId tableId = StatusSection.getTableId(entry.getKey());
-
-        logs.put(file, tableId);
-      }
-    } catch (TableOfflineException e) {
-      log.debug("Replication table isn't online yet");
-    }
-    return logs;
-  }
-
-  private void waitForGCLock(AccumuloClient client) throws InterruptedException {
-    // Check if the GC process has the lock before wasting our retry attempts
-    ZooCache zcache = ((ClientContext) client).getZooCache();
-    var zkPath = ServiceLock
-        .path(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZGC_LOCK);
-    log.info("Looking for GC lock at {}", zkPath);
-    byte[] data = ServiceLock.getLockData(zcache, zkPath, null);
-    while (data == null) {
-      log.info("Waiting for GC ZooKeeper lock to be acquired");
-      Thread.sleep(1000);
-      data = ServiceLock.getLockData(zcache, zkPath, null);
-    }
-  }
-
-  @Test
-  public void replicationTableCreated() {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      assertTrue(client.tableOperations().exists(ReplicationTable.NAME));
-      assertEquals(ReplicationTable.ID.canonical(),
-          client.tableOperations().tableIdMap().get(ReplicationTable.NAME));
-    }
-  }
-
-  @Test
-  public void verifyReplicationTableConfig()
-      throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      TableOperations tops = client.tableOperations();
-      Map<String,EnumSet<IteratorScope>> iterators = tops.listIterators(ReplicationTable.NAME);
-
-      // verify combiners are only iterators (no versioning)
-      assertEquals(1, iterators.size());
-
-      // look for combiner
-      assertTrue(iterators.containsKey(ReplicationTable.COMBINER_NAME));
-      assertTrue(iterators.get(ReplicationTable.COMBINER_NAME)
-          .containsAll(EnumSet.allOf(IteratorScope.class)));
-      for (IteratorScope scope : EnumSet.allOf(IteratorScope.class)) {
-        IteratorSetting is =
-            tops.getIteratorSetting(ReplicationTable.NAME, ReplicationTable.COMBINER_NAME, scope);
-        assertEquals(30, is.getPriority());
-        assertEquals(StatusCombiner.class.getName(), is.getIteratorClass());
-        assertEquals(1, is.getOptions().size());
-        assertTrue(is.getOptions().containsKey("columns"));
-        String cols = is.getOptions().get("columns");
-        Column statusSectionCol = new Column(StatusSection.NAME);
-        Column workSectionCol = new Column(WorkSection.NAME);
-        assertEquals(ColumnSet.encodeColumns(statusSectionCol.getColumnFamily(),
-            statusSectionCol.getColumnQualifier()) + ","
-            + ColumnSet.encodeColumns(workSectionCol.getColumnFamily(),
-                workSectionCol.getColumnQualifier()),
-            cols);
-      }
-
-      boolean foundLocalityGroups = false;
-      boolean foundLocalityGroupDef1 = false;
-      boolean foundLocalityGroupDef2 = false;
-      boolean foundFormatter = false;
-      var joiner = Collectors.joining(",");
-      for (Entry<String,String> p : tops.getProperties(ReplicationTable.NAME)) {
-        String key = p.getKey();
-        String val = p.getValue();
-        // STATUS_LG_NAME, STATUS_LG_COLFAMS, WORK_LG_NAME, WORK_LG_COLFAMS
-        if (key.equals(Property.TABLE_FORMATTER_CLASS.getKey())
-            && val.equals(StatusFormatter.class.getName())) {
-          // look for formatter
-          foundFormatter = true;
-        } else if (key.equals(Property.TABLE_LOCALITY_GROUPS.getKey())
-            && val.equals(ReplicationTable.LOCALITY_GROUPS.keySet().stream().collect(joiner))) {
-          // look for locality groups enabled
-          foundLocalityGroups = true;
-        } else if (key.startsWith(Property.TABLE_LOCALITY_GROUP_PREFIX.getKey())) {
-          // look for locality group column family definitions
-          if (key.equals(
-              Property.TABLE_LOCALITY_GROUP_PREFIX.getKey() + ReplicationTable.STATUS_LG_NAME)
-              && val.equals(ReplicationTable.STATUS_LG_COLFAMS.stream().map(Text::toString)
-                  .collect(joiner))) {
-            foundLocalityGroupDef1 = true;
-          } else if (key
-              .equals(Property.TABLE_LOCALITY_GROUP_PREFIX.getKey() + ReplicationTable.WORK_LG_NAME)
-              && val.equals(
-                  ReplicationTable.WORK_LG_COLFAMS.stream().map(Text::toString).collect(joiner))) {
-            foundLocalityGroupDef2 = true;
-          }
-        }
-      }
-      assertTrue(foundLocalityGroups);
-      assertTrue(foundLocalityGroupDef1);
-      assertTrue(foundLocalityGroupDef2);
-      assertTrue(foundFormatter);
-    }
-  }
-
-  @Test
-  public void correctRecordsCompleteFile() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      String table = "table1";
-      client.tableOperations().create(table, new NewTableConfiguration()
-          // If we have more than one tserver, this is subject to a race condition.
-          .setProperties(singletonMap(Property.TABLE_REPLICATION.getKey(), "true")));
-
-      try (BatchWriter bw = client.createBatchWriter(table)) {
-        for (int i = 0; i < 10; i++) {
-          Mutation m = new Mutation(Integer.toString(i));
-          m.put(new byte[0], new byte[0], new byte[0]);
-          bw.addMutation(m);
-        }
-      }
-
-      // After writing data, we'll get a replication table online
-      while (!ReplicationTable.isOnline(client)) {
-        sleepUninterruptibly(MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS, TimeUnit.MILLISECONDS);
-      }
-      assertTrue(ReplicationTable.isOnline(client), "Replication table did not exist");
-
-      for (int i = 0; i < 5; i++) {
-        if (client.securityOperations().hasTablePermission("root", ReplicationTable.NAME,
-            TablePermission.READ)) {
-          break;
-        }
-        log.info("Could not read replication table, waiting and will retry");
-        Thread.sleep(2000);
-      }
-
-      assertTrue(client.securityOperations().hasTablePermission("root", ReplicationTable.NAME,
-          TablePermission.READ), "'root' user could not read the replication table");
-
-      Set<String> replRows = new HashSet<>();
-      int attempts = 5;
-      while (replRows.isEmpty() && attempts > 0) {
-        try (Scanner scanner = ReplicationTable.getScanner(client)) {
-          StatusSection.limit(scanner);
-          for (Entry<Key,Value> entry : scanner) {
-            Key k = entry.getKey();
-
-            String fileUri = k.getRow().toString();
-            try {
-              new URI(fileUri);
-            } catch (URISyntaxException e) {
-              fail("Expected a valid URI: " + fileUri);
-            }
-
-            replRows.add(fileUri);
-          }
-        }
-      }
-
-      Set<String> wals = new HashSet<>();
-      attempts = 5;
-      while (wals.isEmpty() && attempts > 0) {
-        WalStateManager markers = new WalStateManager(getServerContext());
-        for (Entry<Path,WalState> entry : markers.getAllState().entrySet()) {
-          wals.add(entry.getKey().toString());
-        }
-        attempts--;
-      }
-
-      // We only have one file that should need replication (no trace table)
-      // We should find an entry in tablet and in the repl row
-      assertEquals(1, replRows.size(), "Rows found: " + replRows);
-
-      // There should only be one extra WALog that replication doesn't know about
-      replRows.removeAll(wals);
-      assertEquals(2, wals.size());
-      assertEquals(0, replRows.size());
-    }
-  }
-
-  @Test
-  public void noRecordsWithoutReplication() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      List<String> tables = new ArrayList<>();
-
-      // replication shouldn't be online when we begin
-      assertFalse(ReplicationTable.isOnline(client));
-
-      for (int i = 0; i < 5; i++) {
-        String name = "table" + i;
-        tables.add(name);
-        client.tableOperations().create(name);
-      }
-
-      // nor after we create some tables (that aren't being replicated)
-      assertFalse(ReplicationTable.isOnline(client));
-
-      for (String table : tables) {
-        writeSomeData(client, table, 5, 5);
-      }
-
-      // After writing data, still no replication table
-      assertFalse(ReplicationTable.isOnline(client));
-
-      for (String table : tables) {
-        client.tableOperations().compact(table, null, null, true, true);
-      }
-
-      // After compacting data, still no replication table
-      assertFalse(ReplicationTable.isOnline(client));
-
-      for (String table : tables) {
-        client.tableOperations().delete(table);
-      }
-
-      // After deleting tables, still no replication table
-      assertFalse(ReplicationTable.isOnline(client));
-    }
-  }
-
-  @Test
-  public void twoEntriesForTwoTables() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      String table1 = "table1", table2 = "table2";
-
-      // replication shouldn't exist when we begin
-      assertFalse(ReplicationTable.isOnline(client),
-          "Replication table already online at the beginning of the test");
-
-      // Create two tables
-      client.tableOperations().create(table1);
-      client.tableOperations().create(table2);
-      client.securityOperations().grantTablePermission("root", ReplicationTable.NAME,
-          TablePermission.READ);
-      // wait for permission to propagate
-      Thread.sleep(5000);
-
-      // Enable replication on table1
-      client.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
-
-      // Despite having replication on, we shouldn't have any need to write a record to it (and
-      // bring
-      // it online)
-      assertFalse(ReplicationTable.isOnline(client));
-
-      // Write some data to table1
-      writeSomeData(client, table1, 50, 50);
-
-      // After the commit for these mutations finishes, we'll get a replication entry in
-      // accumulo.metadata for table1
-      // Don't want to compact table1 as it ultimately cause the entry in accumulo.metadata to be
-      // removed before we can verify it's there
-
-      // After writing data, we'll get a replication table online
-      while (!ReplicationTable.isOnline(client)) {
-        sleepUninterruptibly(MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS, TimeUnit.MILLISECONDS);
-      }
-      assertTrue(ReplicationTable.isOnline(client));
-
-      // Verify that we found a single replication record that's for table1
-      Entry<Key,Value> entry;
-      try (Scanner s = ReplicationTable.getScanner(client)) {
-        StatusSection.limit(s);
-        for (int i = 0; i < 5; i++) {
-          if (Iterators.size(s.iterator()) == 1) {
-            break;
-          }
-          Thread.sleep(1000);
-        }
-        entry = Iterators.getOnlyElement(s.iterator());
-      }
-      // We should at least find one status record for this table, we might find a second if another
-      // log was started from ingesting the data
-      assertEquals(client.tableOperations().tableIdMap().get(table1),
-          entry.getKey().getColumnQualifier().toString(),
-          "Expected to find replication entry for " + table1);
-
-      // Enable replication on table2
-      client.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
-
-      // Write some data to table2
-      writeSomeData(client, table2, 50, 50);
-
-      // After the commit on these mutations, we'll get a replication entry in accumulo.metadata for
-      // table2
-      // Don't want to compact table2 as it ultimately cause the entry in accumulo.metadata to be
-      // removed before we can verify it's there
-
-      Set<String> tableIds = Sets.newHashSet(client.tableOperations().tableIdMap().get(table1),
-          client.tableOperations().tableIdMap().get(table2));
-      Set<String> tableIdsForMetadata = Sets.newHashSet(tableIds);
-
-      List<Entry<Key,Value>> records = new ArrayList<>();
-
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        s.setRange(ReplicationSection.getRange());
-        for (Entry<Key,Value> metadata : s) {
-          records.add(metadata);
-          log.debug("Meta: {} => {}", metadata.getKey().toStringNoTruncate(), metadata.getValue());
-        }
-
-        assertEquals(2, records.size(),
-            "Expected to find 2 records, but actually found " + records);
-
-        for (Entry<Key,Value> metadata : records) {
-          assertTrue(tableIdsForMetadata.remove(metadata.getKey().getColumnQualifier().toString()),
-              "Expected record to be in metadata but wasn't "
-                  + metadata.getKey().toStringNoTruncate() + ", tableIds remaining "
-                  + tableIdsForMetadata);
-        }
-
-        assertTrue(tableIdsForMetadata.isEmpty(),
-            "Expected that we had removed all metadata entries " + tableIdsForMetadata);
-
-        // Should be creating these records in replication table from metadata table every second
-        Thread.sleep(5000);
-      }
-
-      // Verify that we found two replication records: one for table1 and one for table2
-      try (Scanner s = ReplicationTable.getScanner(client)) {
-        StatusSection.limit(s);
-        Iterator<Entry<Key,Value>> iter = s.iterator();
-        assertTrue(iter.hasNext(), "Found no records in replication table");
-        entry = iter.next();
-        assertTrue(tableIds.remove(entry.getKey().getColumnQualifier().toString()),
-            "Expected to find element in replication table");
-        assertTrue(iter.hasNext(),
-            "Expected to find two elements in replication table, only found one ");
-        entry = iter.next();
-        assertTrue(tableIds.remove(entry.getKey().getColumnQualifier().toString()),
-            "Expected to find element in replication table");
-        assertFalse(iter.hasNext(), "Expected to only find two elements in replication table");
-      }
-    }
-  }
-
-  private void writeSomeData(AccumuloClient client, String table, int rows, int cols)
-      throws Exception {
-    try (BatchWriter bw = client.createBatchWriter(table)) {
-      for (int row = 0; row < rows; row++) {
-        Mutation m = new Mutation(Integer.toString(row));
-        for (int col = 0; col < cols; col++) {
-          String value = Integer.toString(col);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-    }
-  }
-
-  @Test
-  public void replicationEntriesPrecludeWalDeletion() throws Exception {
-    final ServerContext context = getServerContext();
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      String table1 = "table1", table2 = "table2", table3 = "table3";
-      final Multimap<String,TableId> logs = HashMultimap.create();
-      final AtomicBoolean keepRunning = new AtomicBoolean(true);
-
-      Thread t = new Thread(() -> {
-        // Should really be able to interrupt here, but the Scanner throws a fit to the logger
-        // when that happens
-        while (keepRunning.get()) {
-          try {
-            logs.putAll(getAllLogs(client, context));
-          } catch (Exception e) {
-            log.error("Error getting logs", e);
-          }
-        }
-      });
-
-      t.start();
-      HashMap<String,String> replicate_props = new HashMap<>();
-      replicate_props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      replicate_props.put(Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
-
-      client.tableOperations().create(table1,
-          new NewTableConfiguration().setProperties(replicate_props));
-      Thread.sleep(2000);
-
-      // Write some data to table1
-      writeSomeData(client, table1, 200, 500);
-
-      client.tableOperations().create(table2,
-          new NewTableConfiguration().setProperties(replicate_props));
-      Thread.sleep(2000);
-
-      writeSomeData(client, table2, 200, 500);
-
-      client.tableOperations().create(table3,
-          new NewTableConfiguration().setProperties(replicate_props));
-      Thread.sleep(2000);
-
-      writeSomeData(client, table3, 200, 500);
-
-      // Force a write to metadata for the data written
-      for (String table : Arrays.asList(table1, table2, table3)) {
-        client.tableOperations().flush(table, null, null, true);
-      }
-
-      keepRunning.set(false);
-      t.join(5000);
-
-      // The manager is only running every second to create records in the replication table from
-      // the
-      // metadata table
-      // Sleep a sufficient amount of time to ensure that we get the straggling WALs that might have
-      // been created at the end
-      Thread.sleep(5000);
-
-      Set<String> replFiles = getReferencesToFilesToBeReplicated(client);
-
-      // We might have a WAL that was use solely for the replication table
-      // We want to remove that from our list as it should not appear in the replication table
-      String replicationTableId = client.tableOperations().tableIdMap().get(ReplicationTable.NAME);
-      Iterator<Entry<String,TableId>> observedLogs = logs.entries().iterator();
-      while (observedLogs.hasNext()) {
-        Entry<String,TableId> observedLog = observedLogs.next();
-        if (replicationTableId.equals(observedLog.getValue().canonical())) {
-          log.info("Removing {} because its tableId is for the replication table", observedLog);
-          observedLogs.remove();
-        }
-      }
-
-      // We should have *some* reference to each log that was seen in the metadata table
-      // They might not yet all be closed though (might be newfile)
-      assertTrue(logs.keySet().containsAll(replFiles),
-          "Metadata log distribution: " + logs + "replFiles " + replFiles);
-      assertTrue(logs.keySet().size() - replFiles.size() <= 1,
-          "Difference between replication entries and current logs is bigger than one");
-
-      final Configuration conf = new Configuration();
-      for (String replFile : replFiles) {
-        Path p = new Path(replFile);
-        FileSystem fs = p.getFileSystem(conf);
-        if (!fs.exists(p)) {
-          // double-check: the garbage collector can be fast
-          Set<String> currentSet = getReferencesToFilesToBeReplicated(client);
-          log.info("Current references {}", currentSet);
-          log.info("Looking for reference to {}", replFile);
-          log.info("Contains? {}", currentSet.contains(replFile));
-          assertTrue(!currentSet.contains(replFile),
-              "File does not exist anymore, it was likely incorrectly garbage collected: " + p);
-        }
-      }
-    }
-  }
-
-  private Set<String> getReferencesToFilesToBeReplicated(final AccumuloClient client)
-      throws ReplicationTableOfflineException {
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      StatusSection.limit(s);
-      Set<String> replFiles = new HashSet<>();
-      for (Entry<Key,Value> entry : s) {
-        replFiles.add(entry.getKey().getRow().toString());
-      }
-      return replFiles;
-    }
-  }
-
-  @Test
-  public void combinerWorksOnMetadata() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-
-      client.securityOperations().grantTablePermission("root", MetadataTable.NAME,
-          TablePermission.WRITE);
-
-      ReplicationTableUtil.configureMetadataTable(client, MetadataTable.NAME);
-
-      Status stat1 = StatusUtil.fileCreated(100);
-      Status stat2 = StatusUtil.fileClosed();
-
-      try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
-        Mutation m = new Mutation(
-            ReplicationSection.getRowPrefix() + "file:/accumulo/wals/tserver+port/uuid");
-        m.put(ReplicationSection.COLF, new Text("1"), ProtobufUtil.toValue(stat1));
-        bw.addMutation(m);
-      }
-
-      Status actual;
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        s.setRange(ReplicationSection.getRange());
-
-        actual = Status.parseFrom(getOnlyElement(s).getValue().get());
-        assertEquals(stat1, actual);
-
-        try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
-          Mutation m = new Mutation(
-              ReplicationSection.getRowPrefix() + "file:/accumulo/wals/tserver+port/uuid");
-          m.put(ReplicationSection.COLF, new Text("1"), ProtobufUtil.toValue(stat2));
-          bw.addMutation(m);
-        }
-      }
-
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        s.setRange(ReplicationSection.getRange());
-
-        actual = Status.parseFrom(getOnlyElement(s).getValue().get());
-        Status expected = Status.newBuilder().setBegin(0).setEnd(0).setClosed(true)
-            .setInfiniteEnd(true).setCreatedTime(100).build();
-
-        assertEquals(expected, actual);
-      }
-    }
-  }
-
-  @Test
-  public void noDeadlock() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-
-      ReplicationTable.setOnline(client);
-      client.securityOperations().grantTablePermission("root", ReplicationTable.NAME,
-          TablePermission.WRITE);
-      client.tableOperations().deleteRows(ReplicationTable.NAME, null, null);
-
-      Map<String,String> replicate_props = new HashMap<>();
-      replicate_props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      replicate_props.put(Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
-
-      String table1 = "table1", table2 = "table2", table3 = "table3";
-      client.tableOperations().create(table1,
-          new NewTableConfiguration().setProperties(replicate_props));
-
-      client.tableOperations().create(table2,
-          new NewTableConfiguration().setProperties(replicate_props));
-
-      client.tableOperations().create(table3,
-          new NewTableConfiguration().setProperties(replicate_props));
-
-      writeSomeData(client, table1, 200, 500);
-
-      writeSomeData(client, table2, 200, 500);
-
-      writeSomeData(client, table3, 200, 500);
-
-      // Flush everything to try to make the replication records
-      for (String table : Arrays.asList(table1, table2, table3)) {
-        client.tableOperations().flush(table, null, null, true);
-      }
-
-      // Flush everything to try to make the replication records
-      for (String table : Arrays.asList(table1, table2, table3)) {
-        client.tableOperations().flush(table, null, null, true);
-      }
-
-      for (String table : Arrays.asList(MetadataTable.NAME, table1, table2, table3)) {
-        try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
-          scanner.forEach((k, v) -> {});
-        }
-      }
-    }
-  }
-
-  @Test
-  public void filesClosedAfterUnused() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-
-      String table = "table";
-      Map<String,String> replicate_props = new HashMap<>();
-      replicate_props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      replicate_props.put(Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
-
-      client.tableOperations().create(table,
-          new NewTableConfiguration().setProperties(replicate_props));
-      TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(table));
-
-      assertNotNull(tableId);
-
-      // just sleep
-      client.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-          ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "50000"));
-
-      // Write a mutation to make a log file
-      try (BatchWriter bw = client.createBatchWriter(table)) {
-        Mutation m = new Mutation("one");
-        m.put("", "", "");
-        bw.addMutation(m);
-      }
-
-      // Write another to make sure the logger rolls itself?
-      try (BatchWriter bw = client.createBatchWriter(table)) {
-        Mutation m = new Mutation("three");
-        m.put("", "", "");
-        bw.addMutation(m);
-      }
-
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        s.fetchColumnFamily(LogColumnFamily.NAME);
-        s.setRange(TabletsSection.getRange(tableId));
-        Set<String> wals = new HashSet<>();
-        for (Entry<Key,Value> entry : s) {
-          LogEntry logEntry = LogEntry.fromMetaWalEntry(entry);
-          wals.add(new Path(logEntry.filename).toString());
-        }
-
-        log.warn("Found wals {}", wals);
-
-        try (BatchWriter bw = client.createBatchWriter(table)) {
-          Mutation m = new Mutation("three");
-          byte[] bytes = new byte[1024 * 1024];
-          m.put("1".getBytes(), new byte[0], bytes);
-          m.put("2".getBytes(), new byte[0], bytes);
-          m.put("3".getBytes(), new byte[0], bytes);
-          m.put("4".getBytes(), new byte[0], bytes);
-          m.put("5".getBytes(), new byte[0], bytes);
-          bw.addMutation(m);
-        }
-
-        client.tableOperations().flush(table, null, null, true);
-
-        while (!ReplicationTable.isOnline(client)) {
-          sleepUninterruptibly(MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS,
-              TimeUnit.MILLISECONDS);
-        }
-
-        for (int i = 0; i < 10; i++) {
-          try (Scanner s2 = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-            s2.fetchColumnFamily(LogColumnFamily.NAME);
-            s2.setRange(TabletsSection.getRange(tableId));
-            for (Entry<Key,Value> entry : s2) {
-              log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-            }
-          }
-
-          try (Scanner s3 = ReplicationTable.getScanner(client)) {
-            StatusSection.limit(s3);
-            Text buff = new Text();
-            boolean allReferencedLogsClosed = true;
-            int recordsFound = 0;
-            for (Entry<Key,Value> e : s3) {
-              recordsFound++;
-              allReferencedLogsClosed = true;
-              StatusSection.getFile(e.getKey(), buff);
-              String file = buff.toString();
-              if (wals.contains(file)) {
-                Status stat = Status.parseFrom(e.getValue().get());
-                if (!stat.getClosed()) {
-                  log.info("{} wasn't closed", file);
-                  allReferencedLogsClosed = false;
-                }
-              }
-            }
-
-            if (recordsFound > 0 && allReferencedLogsClosed) {
-              return;
-            }
-            Thread.sleep(2000);
-          } catch (RuntimeException e) {
-            Throwable cause = e.getCause();
-            if (cause instanceof AccumuloSecurityException) {
-              AccumuloSecurityException ase = (AccumuloSecurityException) cause;
-              switch (ase.getSecurityErrorCode()) {
-                case PERMISSION_DENIED:
-                  // We tried to read the replication table before the GRANT went through
-                  Thread.sleep(2000);
-                  break;
-                default:
-                  throw e;
-              }
-            }
-          }
-        }
-        fail("We had a file that was referenced but didn't get closed");
-      }
-    }
-  }
-
-  @Test
-  public void singleTableWithSingleTarget() throws Exception {
-    // We want to kill the GC so it doesn't come along and close Status records and mess up the
-    // comparisons
-    // against expected Status messages.
-    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
-
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      String table1 = "table1";
-
-      // replication shouldn't be online when we begin
-      assertFalse(ReplicationTable.isOnline(client));
-
-      // Create a table
-      client.tableOperations().create(table1);
-
-      int attempts = 10;
-
-      // Might think the table doesn't yet exist, retry
-      while (attempts > 0) {
-        try {
-          // Enable replication on table1
-          client.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
-          // Replicate table1 to cluster1 in the table with id of '4'
-          client.tableOperations().setProperty(table1,
-              Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "4");
-          // Sleep for 100 seconds before saying something is replicated
-          client.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-              ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "100000"));
-          break;
-        } catch (Exception e) {
-          attempts--;
-          if (attempts <= 0) {
-            throw e;
-          }
-          sleepUninterruptibly(2, TimeUnit.SECONDS);
-        }
-      }
-
-      // Write some data to table1
-      writeSomeData(client, table1, 2000, 50);
-
-      // Make sure the replication table is online at this point
-      while (!ReplicationTable.isOnline(client)) {
-        sleepUninterruptibly(MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS, TimeUnit.MILLISECONDS);
-      }
-      assertTrue(ReplicationTable.isOnline(client), "Replication table was never created");
-
-      // ACCUMULO-2743 The Observer in the tserver has to be made aware of the change to get the
-      // combiner (made by the manager)
-      for (int i = 0; i < 10 && !client.tableOperations().listIterators(ReplicationTable.NAME)
-          .containsKey(ReplicationTable.COMBINER_NAME); i++) {
-        sleepUninterruptibly(2, TimeUnit.SECONDS);
-      }
-
-      assertTrue(client.tableOperations().listIterators(ReplicationTable.NAME).containsKey(
-          ReplicationTable.COMBINER_NAME), "Combiner was never set on replication table");
-
-      // Trigger the minor compaction, waiting for it to finish.
-      // This should write the entry to metadata that the file has data
-      client.tableOperations().flush(table1, null, null, true);
-
-      // Make sure that we have one status element, should be a new file
-      try (Scanner s = ReplicationTable.getScanner(client)) {
-        StatusSection.limit(s);
-        Entry<Key,Value> entry = null;
-        Status expectedStatus = StatusUtil.openWithUnknownLength();
-        attempts = 10;
-        // This record will move from new to new with infinite length because of the minc (flush)
-        while (entry == null && attempts > 0) {
-          try {
-            entry = getOnlyElement(s);
-            Status actual = Status.parseFrom(entry.getValue().get());
-            if (actual.getInfiniteEnd() != expectedStatus.getInfiniteEnd()) {
-              entry = null;
-              // the manager process didn't yet fire and write the new mutation, wait for it to do
-              // so and try to read it again
-              Thread.sleep(1000);
-            }
-          } catch (NoSuchElementException e) {
-            entry = null;
-            Thread.sleep(500);
-          } catch (IllegalArgumentException e) {
-            // saw this contain 2 elements once
-            try (Scanner s2 = ReplicationTable.getScanner(client)) {
-              StatusSection.limit(s2);
-              for (Entry<Key,Value> content : s2) {
-                log.info("{} => {}", content.getKey().toStringNoTruncate(), content.getValue());
-              }
-              throw e;
-            }
-          } finally {
-            attempts--;
-          }
-        }
-
-        assertNotNull(entry, "Could not find expected entry in replication table");
-        Status actual = Status.parseFrom(entry.getValue().get());
-        assertTrue(!actual.getClosed() && actual.getInfiniteEnd(),
-            "Expected to find a replication entry that is open with infinite length: "
-                + ProtobufUtil.toString(actual));
-
-        // Try a couple of times to watch for the work record to be created
-        boolean notFound = true;
-        for (int i = 0; i < 10 && notFound; i++) {
-          try (Scanner s2 = ReplicationTable.getScanner(client)) {
-            WorkSection.limit(s2);
-            int elementsFound = Iterables.size(s2);
-            if (elementsFound > 0) {
-              assertEquals(1, elementsFound);
-              notFound = false;
-            }
-            Thread.sleep(500);
-          }
-        }
-
-        // If we didn't find the work record, print the contents of the table
-        if (notFound) {
-          try (Scanner s2 = ReplicationTable.getScanner(client)) {
-            for (Entry<Key,Value> content : s2) {
-              log.info("{} => {}", content.getKey().toStringNoTruncate(), content.getValue());
-            }
-            assertFalse(notFound, "Did not find the work entry for the status entry");
-          }
-        }
-
-        // Write some more data so that we over-run the single WAL
-        writeSomeData(client, table1, 3000, 50);
-
-        log.info("Issued compaction for table");
-        client.tableOperations().compact(table1, null, null, true, true);
-        log.info("Compaction completed");
-
-        // Manager is creating entries in the replication table from the metadata table every
-        // second.
-        // Compaction should trigger the record to be written to metadata. Wait a bit to ensure
-        // that the manager has time to work.
-        Thread.sleep(5000);
-
-        try (Scanner s2 = ReplicationTable.getScanner(client)) {
-          StatusSection.limit(s2);
-          int numRecords = 0;
-          for (Entry<Key,Value> e : s2) {
-            numRecords++;
-            log.info("Found status record {}\t{}", e.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(e.getValue().get())));
-          }
-
-          assertEquals(2, numRecords);
-        }
-
-        // We should eventually get 2 work records recorded, need to account for a potential delay
-        // though
-        // might see: status1 -> work1 -> status2 -> (our scans) -> work2
-        notFound = true;
-        for (int i = 0; i < 10 && notFound; i++) {
-          try (Scanner s2 = ReplicationTable.getScanner(client)) {
-            WorkSection.limit(s2);
-            int elementsFound = Iterables.size(s2);
-            if (elementsFound == 2) {
-              notFound = false;
-            }
-            Thread.sleep(500);
-          }
-        }
-
-        // If we didn't find the work record, print the contents of the table
-        if (notFound) {
-          try (Scanner s2 = ReplicationTable.getScanner(client)) {
-            for (Entry<Key,Value> content : s2) {
-              log.info("{} => {}", content.getKey().toStringNoTruncate(), content.getValue());
-            }
-            assertFalse(notFound, "Did not find the work entries for the status entries");
-          }
-        }
-      }
-    }
-  }
-
-  @Test
-  public void correctClusterNameInWorkEntry() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      String table1 = "table1";
-
-      // replication shouldn't be online when we begin
-      assertFalse(ReplicationTable.isOnline(client));
-
-      // Create two tables
-      client.tableOperations().create(table1);
-
-      int attempts = 5;
-      while (attempts > 0) {
-        try {
-          // Enable replication on table1
-          client.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
-          // Replicate table1 to cluster1 in the table with id of '4'
-          client.tableOperations().setProperty(table1,
-              Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "4");
-          attempts = 0;
-        } catch (Exception e) {
-          attempts--;
-          if (attempts <= 0) {
-            throw e;
-          }
-          sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
-        }
-      }
-
-      // Write some data to table1
-      writeSomeData(client, table1, 2000, 50);
-      client.tableOperations().flush(table1, null, null, true);
-
-      TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(table1));
-      assertNotNull(tableId, "Table ID was null");
-
-      // Make sure the replication table exists at this point
-      while (!ReplicationTable.isOnline(client)) {
-        sleepUninterruptibly(MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS, TimeUnit.MILLISECONDS);
-      }
-      assertTrue(ReplicationTable.isOnline(client), "Replication table did not exist");
-
-      for (int i = 0; i < 5 && !client.securityOperations().hasTablePermission("root",
-          ReplicationTable.NAME, TablePermission.READ); i++) {
-        Thread.sleep(1000);
-      }
-
-      assertTrue(client.securityOperations().hasTablePermission("root", ReplicationTable.NAME,
-          TablePermission.READ));
-
-      boolean notFound = true;
-      for (int i = 0; i < 10 && notFound; i++) {
-        try (Scanner s = ReplicationTable.getScanner(client)) {
-          WorkSection.limit(s);
-          try {
-            Entry<Key,Value> e = getOnlyElement(s);
-            Text expectedColqual = new ReplicationTarget("cluster1", "4", tableId).toText();
-            assertEquals(expectedColqual, e.getKey().getColumnQualifier());
-            notFound = false;
-          } catch (NoSuchElementException e) {} catch (IllegalArgumentException e) {
-            try (Scanner s2 = ReplicationTable.getScanner(client)) {
-              for (Entry<Key,Value> content : s2) {
-                log.info("{} => {}", content.getKey().toStringNoTruncate(), content.getValue());
-              }
-              fail("Found more than one work section entry");
-            }
-          }
-          Thread.sleep(500);
-        }
-      }
-
-      if (notFound) {
-        try (Scanner s = ReplicationTable.getScanner(client)) {
-          for (Entry<Key,Value> content : s) {
-            log.info("{} => {}", content.getKey().toStringNoTruncate(), content.getValue());
-          }
-          assertFalse(notFound, "Did not find the work entry for the status entry");
-        }
-      }
-    }
-  }
-
-  @Test
-  public void replicationRecordsAreClosedAfterGarbageCollection() throws Exception {
-    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
-
-    final ServerContext context = getServerContext();
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-
-      ReplicationTable.setOnline(client);
-      client.securityOperations().grantTablePermission("root", ReplicationTable.NAME,
-          TablePermission.WRITE);
-      client.tableOperations().deleteRows(ReplicationTable.NAME, null, null);
-
-      final AtomicBoolean keepRunning = new AtomicBoolean(true);
-      final Set<String> metadataWals = new HashSet<>();
-
-      Thread t = new Thread(() -> {
-        // Should really be able to interrupt here, but the Scanner throws a fit to the logger
-        // when that happens
-        while (keepRunning.get()) {
-          try {
-            metadataWals.addAll(getLogs(client, context).keySet());
-          } catch (Exception e) {
-            log.error("Metadata table doesn't exist");
-          }
-        }
-      });
-
-      t.start();
-
-      String table1 = "table1", table2 = "table2", table3 = "table3";
-      Map<String,String> replicate_props = new HashMap<>();
-      replicate_props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      replicate_props.put(Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
-
-      try {
-        client.tableOperations().create(table1,
-            new NewTableConfiguration().setProperties(replicate_props));
-
-        client.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-            ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, null));
-
-        // Write some data to table1
-        writeSomeData(client, table1, 200, 500);
-
-        client.tableOperations().create(table2,
-            new NewTableConfiguration().setProperties(replicate_props));
-
-        writeSomeData(client, table2, 200, 500);
-
-        client.tableOperations().create(table3,
-            new NewTableConfiguration().setProperties(replicate_props));
-
-        writeSomeData(client, table3, 200, 500);
-
-        // Flush everything to try to make the replication records
-        for (String table : Arrays.asList(table1, table2, table3)) {
-          client.tableOperations().compact(table, null, null, true, true);
-        }
-      } finally {
-        keepRunning.set(false);
-        t.join(5000);
-        assertFalse(t.isAlive());
-      }
-
-      // Kill the tserver(s) and restart them
-      // to ensure that the WALs we previously observed all move to closed.
-      cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
-      cluster.getClusterControl().start(ServerType.TABLET_SERVER);
-
-      // Make sure we can read all the tables (recovery complete)
-      for (String table : Arrays.asList(table1, table2, table3)) {
-        try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
-          scanner.forEach((k, v) -> {});
-        }
-      }
-
-      // Starting the gc will run CloseWriteAheadLogReferences which will first close Statuses
-      // in the metadata table, and then in the replication table
-      Process gc = cluster.exec(SimpleGarbageCollector.class).getProcess();
-
-      waitForGCLock(client);
-
-      Thread.sleep(1000);
-
-      log.info("GC is up and should have had time to run at least once by now");
-
-      try {
-        boolean allClosed = true;
-
-        // We should either find all closed records or no records
-        // After they're closed, they are candidates for deletion
-        for (int i = 0; i < 10; i++) {
-          try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-            s.setRange(Range.prefix(ReplicationSection.getRowPrefix()));
-            Iterator<Entry<Key,Value>> iter = s.iterator();
-
-            long recordsFound = 0L;
-            while (allClosed && iter.hasNext()) {
-              Entry<Key,Value> entry = iter.next();
-              String wal = entry.getKey().getRow().toString();
-              if (metadataWals.contains(wal)) {
-                Status status = Status.parseFrom(entry.getValue().get());
-                log.info("{}={}", entry.getKey().toStringNoTruncate(),
-                    ProtobufUtil.toString(status));
-                allClosed &= status.getClosed();
-                recordsFound++;
-              }
-            }
-
-            log.info("Found {} records from the metadata table", recordsFound);
-            if (allClosed) {
-              break;
-            }
-            sleepUninterruptibly(2, TimeUnit.SECONDS);
-          }
-        }
-
-        if (!allClosed) {
-          try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-            s.setRange(Range.prefix(ReplicationSection.getRowPrefix()));
-            for (Entry<Key,Value> entry : s) {
-              log.info("{} {}", entry.getKey().toStringNoTruncate(),
-                  ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
-            }
-            fail("Expected all replication records in the metadata table to be closed");
-          }
-        }
-
-        for (int i = 0; i < 10; i++) {
-          allClosed = true;
-
-          try (Scanner s = ReplicationTable.getScanner(client)) {
-            Iterator<Entry<Key,Value>> iter = s.iterator();
-
-            long recordsFound = 0L;
-            while (allClosed && iter.hasNext()) {
-              Entry<Key,Value> entry = iter.next();
-              String wal = entry.getKey().getRow().toString();
-              if (metadataWals.contains(wal)) {
-                Status status = Status.parseFrom(entry.getValue().get());
-                log.info("{}={}", entry.getKey().toStringNoTruncate(),
-                    ProtobufUtil.toString(status));
-                allClosed &= status.getClosed();
-                recordsFound++;
-              }
-            }
-
-            log.info("Found {} records from the replication table", recordsFound);
-            if (allClosed) {
-              break;
-            }
-            sleepUninterruptibly(3, TimeUnit.SECONDS);
-          }
-        }
-
-        if (!allClosed) {
-          try (Scanner s = ReplicationTable.getScanner(client)) {
-            StatusSection.limit(s);
-            for (Entry<Key,Value> entry : s) {
-              log.info("{} {}", entry.getKey().toStringNoTruncate(),
-                  TextFormat.shortDebugString(Status.parseFrom(entry.getValue().get())));
-            }
-            fail("Expected all replication records in the replication table to be closed");
-          }
-        }
-
-      } finally {
-        gc.destroy();
-        gc.waitFor();
-      }
-
-    }
-  }
-
-  @Test
-  public void replicatedStatusEntriesAreDeleted() throws Exception {
-    // Just stop it now, we'll restart it after we restart the tserver
-    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
-
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      log.info("Got client to MAC");
-      String table1 = "table1";
-
-      // replication shouldn't be online when we begin
-      assertFalse(ReplicationTable.isOnline(client));
-
-      // Create two tables
-      client.tableOperations().create(table1);
-
-      int attempts = 5;
-      while (attempts > 0) {
-        try {
-          // Enable replication on table1
-          client.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
-          // Replicate table1 to cluster1 in the table with id of '4'
-          client.tableOperations().setProperty(table1,
-              Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "4");
-          // Use the MockReplicaSystem impl and sleep for 5seconds
-          client.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-              ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "1000"));
-          attempts = 0;
-        } catch (Exception e) {
-          attempts--;
-          if (attempts <= 0) {
-            throw e;
-          }
-          sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
-        }
-      }
-
-      TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(table1));
-      assertNotNull(tableId, "Could not determine table id for " + table1);
-
-      // Write some data to table1
-      writeSomeData(client, table1, 2000, 50);
-      client.tableOperations().flush(table1, null, null, true);
-
-      // Make sure the replication table exists at this point
-      while (!ReplicationTable.isOnline(client)) {
-        sleepUninterruptibly(MILLIS_BETWEEN_REPLICATION_TABLE_ONLINE_CHECKS, TimeUnit.MILLISECONDS);
-      }
-      assertTrue(ReplicationTable.isOnline(client), "Replication table did not exist");
-
-      // Grant ourselves the write permission for later
-      client.securityOperations().grantTablePermission("root", ReplicationTable.NAME,
-          TablePermission.WRITE);
-
-      log.info("Checking for replication entries in replication");
-      // Then we need to get those records over to the replication table
-      Set<String> entries = new HashSet<>();
-      for (int i = 0; i < 5; i++) {
-        try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-          s.setRange(ReplicationSection.getRange());
-          entries.clear();
-          for (Entry<Key,Value> entry : s) {
-            entries.add(entry.getKey().getRow().toString());
-            log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-          }
-          if (!entries.isEmpty()) {
-            log.info("Replication entries {}", entries);
-            break;
-          }
-          Thread.sleep(1000);
-        }
-      }
-
-      assertFalse(entries.isEmpty(),
-          "Did not find any replication entries in the replication table");
-
-      // Find the WorkSection record that will be created for that data we ingested
-      boolean notFound = true;
-      for (int i = 0; i < 10 && notFound; i++) {
-        try (Scanner s = ReplicationTable.getScanner(client)) {
-          WorkSection.limit(s);
-          Entry<Key,Value> e = getOnlyElement(s);
-          log.info("Found entry: {}", e.getKey().toStringNoTruncate());
-          Text expectedColqual = new ReplicationTarget("cluster1", "4", tableId).toText();
-          assertEquals(expectedColqual, e.getKey().getColumnQualifier());
-          notFound = false;
-        } catch (NoSuchElementException e) {
-
-        } catch (IllegalArgumentException e) {
-          // Somehow we got more than one element. Log what they were
-          try (Scanner s = ReplicationTable.getScanner(client)) {
-            for (Entry<Key,Value> content : s) {
-              log.info("{} => {}", content.getKey().toStringNoTruncate(), content.getValue());
-            }
-            fail("Found more than one work section entry");
-          }
-        } catch (RuntimeException e) {
-          // Catch a propagation issue, fail if it's not what we expect
-          Throwable cause = e.getCause();
-          if (cause instanceof AccumuloSecurityException) {
-            AccumuloSecurityException sec = (AccumuloSecurityException) cause;
-            switch (sec.getSecurityErrorCode()) {
-              case PERMISSION_DENIED:
-                // retry -- the grant didn't happen yet
-                log.warn("Sleeping because permission was denied");
-                break;
-              default:
-                throw e;
-            }
-          } else {
-            throw e;
-          }
-        }
-        Thread.sleep(2000);
-      }
-
-      if (notFound) {
-        try (Scanner s = ReplicationTable.getScanner(client)) {
-          for (Entry<Key,Value> content : s) {
-            log.info("{} => {}", content.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(content.getValue().get())));
-          }
-          assertFalse(notFound, "Did not find the work entry for the status entry");
-        }
-      }
-
-      /**
-       * By this point, we should have data ingested into a table, with at least one WAL as a
-       * candidate for replication. Compacting the table should close all open WALs, which should
-       * ensure all records we're going to replicate have entries in the replication table, and
-       * nothing will exist in the metadata table anymore
-       */
-
-      log.info("Killing tserver");
-      // Kill the tserver(s) and restart them
-      // to ensure that the WALs we previously observed all move to closed.
-      cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
-
-      log.info("Starting tserver");
-      cluster.getClusterControl().start(ServerType.TABLET_SERVER);
-
-      log.info("Waiting to read tables");
-      sleepUninterruptibly(2 * 3, TimeUnit.SECONDS);
-
-      // Make sure we can read all the tables (recovery complete)
-      for (String table : new String[] {MetadataTable.NAME, table1}) {
-        try (Scanner scanner = client.createScanner(table, Authorizations.EMPTY)) {
-          scanner.forEach((k, v) -> {});
-        }
-      }
-
-      log.info("Recovered metadata:");
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> entry : s) {
-          log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-        }
-      }
-
-      cluster.getClusterControl().start(ServerType.GARBAGE_COLLECTOR);
-
-      // Wait for a bit since the GC has to run (should be running after a one second delay)
-      waitForGCLock(client);
-
-      Thread.sleep(1000);
-
-      log.info("After GC");
-      try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> entry : s) {
-          log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-        }
-      }
-
-      // We expect no records in the metadata table after compaction. We have to poll
-      // because we have to wait for the StatusMaker's next iteration which will clean
-      // up the dangling *closed* records after we create the record in the replication table.
-      // We need the GC to close the file (CloseWriteAheadLogReferences) before we can remove the
-      // record
-      log.info("Checking metadata table for replication entries");
-      Set<String> remaining = new HashSet<>();
-      for (int i = 0; i < 10; i++) {
-        try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-          s.setRange(ReplicationSection.getRange());
-          remaining.clear();
-          for (Entry<Key,Value> e : s) {
-            remaining.add(e.getKey().getRow().toString());
-          }
-          remaining.retainAll(entries);
-          if (remaining.isEmpty()) {
-            break;
-          }
-          log.info("remaining {}", remaining);
-          Thread.sleep(2000);
-          log.info("");
-        }
-      }
-
-      assertTrue(remaining.isEmpty(),
-          "Replication status messages were not cleaned up from metadata table");
-
-      /**
-       * After we close out and subsequently delete the metadata record, this will propagate to the
-       * replication table, which will cause those records to be deleted after replication occurs
-       */
-
-      int recordsFound = 0;
-      for (int i = 0; i < 30; i++) {
-        try (Scanner s = ReplicationTable.getScanner(client)) {
-          recordsFound = 0;
-          for (Entry<Key,Value> entry : s) {
-            recordsFound++;
-            log.info("{} {}", entry.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
-          }
-
-          if (recordsFound <= 2) {
-            break;
-          } else {
-            Thread.sleep(1000);
-            log.info("");
-          }
-        }
-      }
-      assertTrue(recordsFound <= 2,
-          "Found unexpected replication records in the replication table");
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
deleted file mode 100644
index e84a137..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
+++ /dev/null
@@ -1,455 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ReplicationOperationsImpl;
-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.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.manager.Manager;
-import org.apache.accumulo.manager.ManagerClientServiceHandler;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.io.Text;
-import org.apache.thrift.TException;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class ReplicationOperationsImplIT extends ConfigurableMacBase {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImplIT.class);
-
-  private AccumuloClient client;
-  private ServerContext context;
-
-  @BeforeEach
-  public void configureInstance() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    context = getServerContext();
-    ReplicationTable.setOnline(client);
-    client.securityOperations().grantTablePermission(client.whoami(), MetadataTable.NAME,
-        TablePermission.WRITE);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-  }
-
-  /**
-   * Spoof out the Manager so we can call the implementation without starting a full instance.
-   */
-  private ReplicationOperationsImpl getReplicationOperations() {
-    Manager manager = EasyMock.createMock(Manager.class);
-    EasyMock.expect(manager.getContext()).andReturn(context).anyTimes();
-    EasyMock.replay(manager);
-
-    final ManagerClientServiceHandler mcsh = new ManagerClientServiceHandler(manager) {
-      @Override
-      protected TableId getTableId(ClientContext context, String tableName) {
-        try {
-          return TableId.of(client.tableOperations().tableIdMap().get(tableName));
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-
-    ClientContext context = (ClientContext) client;
-    return new ReplicationOperationsImpl(context) {
-      @Override
-      protected boolean getManagerDrain(final TInfo tinfo, final TCredentials rpcCreds,
-          final String tableName, final Set<String> wals) {
-        try {
-          return mcsh.drainReplicationTable(tinfo, rpcCreds, tableName, wals);
-        } catch (TException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  @Test
-  public void waitsUntilEntriesAreReplicated() throws Exception {
-    client.tableOperations().create("foo");
-    TableId tableId = TableId.of(client.tableOperations().tableIdMap().get("foo"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(),
-        file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false)
-        .setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    bw.close();
-
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, new Text(tableId.canonical()), ProtobufUtil.toValue(stat));
-
-    bw.addMutation(m);
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
-    m.put(ReplicationSection.COLF, new Text(tableId.canonical()), ProtobufUtil.toValue(stat));
-
-    bw.close();
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    final ReplicationOperationsImpl roi = getReplicationOperations();
-    Thread t = new Thread(() -> {
-      try {
-        roi.drain("foo");
-      } catch (Exception e) {
-        log.error("Got error", e);
-        exception.set(true);
-      }
-      done.set(true);
-    });
-
-    t.start();
-
-    // With the records, we shouldn't be drained
-    assertFalse(done.get());
-
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.putDelete(ReplicationSection.COLF, new Text(tableId.canonical()));
-    bw.addMutation(m);
-    bw.flush();
-
-    assertFalse(done.get());
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
-    m.putDelete(ReplicationSection.COLF, new Text(tableId.canonical()));
-    bw.addMutation(m);
-    bw.flush();
-    bw.close();
-
-    // Removing metadata entries doesn't change anything
-    assertFalse(done.get());
-
-    // Remove the replication entries too
-    bw = ReplicationTable.getBatchWriter(client);
-    m = new Mutation(file1);
-    m.putDelete(StatusSection.NAME, new Text(tableId.canonical()));
-    bw.addMutation(m);
-    bw.flush();
-
-    assertFalse(done.get());
-
-    m = new Mutation(file2);
-    m.putDelete(StatusSection.NAME, new Text(tableId.canonical()));
-    bw.addMutation(m);
-    bw.flush();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      fail("ReplicationOperations.drain did not complete");
-    }
-
-    // After both metadata and replication
-    assertTrue(done.get(), "Drain never finished");
-    assertFalse(exception.get(), "Saw unexpected exception");
-  }
-
-  @Test
-  public void unrelatedReplicationRecordsDontBlockDrain() throws Exception {
-    client.tableOperations().create("foo");
-    client.tableOperations().create("bar");
-
-    TableId tableId1 = TableId.of(client.tableOperations().tableIdMap().get("foo"));
-    TableId tableId2 = TableId.of(client.tableOperations().tableIdMap().get("bar"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(),
-        file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false)
-        .setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    StatusSection.add(m, tableId2, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    bw.close();
-
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, new Text(tableId1.canonical()), ProtobufUtil.toValue(stat));
-
-    bw.addMutation(m);
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
-    m.put(ReplicationSection.COLF, new Text(tableId2.canonical()), ProtobufUtil.toValue(stat));
-
-    bw.close();
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-
-    final ReplicationOperationsImpl roi = getReplicationOperations();
-
-    Thread t = new Thread(() -> {
-      try {
-        roi.drain("foo");
-      } catch (Exception e) {
-        log.error("Got error", e);
-        exception.set(true);
-      }
-      done.set(true);
-    });
-
-    t.start();
-
-    // With the records, we shouldn't be drained
-    assertFalse(done.get());
-
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.putDelete(ReplicationSection.COLF, new Text(tableId1.canonical()));
-    bw.addMutation(m);
-    bw.flush();
-
-    // Removing metadata entries doesn't change anything
-    assertFalse(done.get());
-
-    // Remove the replication entries too
-    bw = ReplicationTable.getBatchWriter(client);
-    m = new Mutation(file1);
-    m.putDelete(StatusSection.NAME, new Text(tableId1.canonical()));
-    bw.addMutation(m);
-    bw.flush();
-    bw.close();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      fail("ReplicationOperations.drain did not complete");
-    }
-
-    // After both metadata and replication
-    assertTrue(done.get(), "Drain never completed");
-    assertFalse(exception.get(), "Saw unexpected exception");
-  }
-
-  @Test
-  public void inprogressReplicationRecordsBlockExecution() throws Exception {
-    client.tableOperations().create("foo");
-
-    TableId tableId1 = TableId.of(client.tableOperations().tableIdMap().get("foo"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false)
-        .setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    bw.close();
-
-    LogEntry logEntry =
-        new LogEntry(new KeyExtent(tableId1, null, null), System.currentTimeMillis(), file1);
-
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, new Text(tableId1.canonical()), ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    m = new Mutation(logEntry.getRow());
-    m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
-    bw.addMutation(m);
-
-    bw.close();
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    final ReplicationOperationsImpl roi = getReplicationOperations();
-    Thread t = new Thread(() -> {
-      try {
-        roi.drain("foo");
-      } catch (Exception e) {
-        log.error("Got error", e);
-        exception.set(true);
-      }
-      done.set(true);
-    });
-
-    t.start();
-
-    // With the records, we shouldn't be drained
-    assertFalse(done.get());
-
-    Status newStatus = Status.newBuilder().setBegin(1000).setEnd(2000).setInfiniteEnd(false)
-        .setClosed(true).build();
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, new Text(tableId1.canonical()), ProtobufUtil.toValue(newStatus));
-    bw.addMutation(m);
-    bw.flush();
-
-    // Removing metadata entries doesn't change anything
-    assertFalse(done.get());
-
-    // Remove the replication entries too
-    bw = ReplicationTable.getBatchWriter(client);
-    m = new Mutation(file1);
-    m.put(StatusSection.NAME, new Text(tableId1.canonical()), ProtobufUtil.toValue(newStatus));
-    bw.addMutation(m);
-    bw.flush();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      fail("ReplicationOperations.drain did not complete");
-    }
-
-    // New records, but not fully replicated ones don't cause it to complete
-    assertFalse(done.get(), "Drain somehow finished");
-    assertFalse(exception.get(), "Saw unexpected exception");
-  }
-
-  @Test
-  public void laterCreatedLogsDontBlockExecution() throws Exception {
-    client.tableOperations().create("foo");
-
-    TableId tableId1 = TableId.of(client.tableOperations().tableIdMap().get("foo"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false)
-        .setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    bw.close();
-
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, new Text(tableId1.canonical()), ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    bw.close();
-
-    log.info("Reading metadata first time");
-    try (var scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      for (Entry<Key,Value> e : scanner) {
-        log.info("{}", e.getKey());
-      }
-    }
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    final ReplicationOperationsImpl roi = getReplicationOperations();
-    Thread t = new Thread(() -> {
-      try {
-        roi.drain("foo");
-      } catch (Exception e) {
-        log.error("Got error", e);
-        exception.set(true);
-      }
-      done.set(true);
-    });
-
-    t.start();
-
-    // We need to wait long enough for the table to read once
-    Thread.sleep(2000);
-
-    // Write another file, but also delete the old files
-    bw = client.createBatchWriter(MetadataTable.NAME);
-    m = new Mutation(
-        ReplicationSection.getRowPrefix() + "/accumulo/wals/tserver+port/" + UUID.randomUUID());
-    m.put(ReplicationSection.COLF, new Text(tableId1.canonical()), ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.putDelete(ReplicationSection.COLF, new Text(tableId1.canonical()));
-    bw.addMutation(m);
-    bw.close();
-
-    log.info("Reading metadata second time");
-    try (var scanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      for (Entry<Key,Value> e : scanner) {
-        log.info("{}", e.getKey());
-      }
-    }
-
-    bw = ReplicationTable.getBatchWriter(client);
-    m = new Mutation(file1);
-    m.putDelete(StatusSection.NAME, new Text(tableId1.canonical()));
-    bw.addMutation(m);
-    bw.close();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      fail("ReplicationOperations.drain did not complete");
-    }
-
-    // We should pass immediately because we aren't waiting on both files to be deleted (just the
-    // one that we did)
-    assertTrue(done.get(), "Drain didn't finish");
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java
deleted file mode 100644
index e49a967..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java
+++ /dev/null
@@ -1,401 +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.test.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.SequentialWorkAssigner;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class SequentialWorkAssignerIT extends ConfigurableMacBase {
-
-  private AccumuloClient client;
-  private MockSequentialWorkAssigner assigner;
-
-  private static class MockSequentialWorkAssigner extends SequentialWorkAssigner {
-
-    public MockSequentialWorkAssigner(AccumuloClient client) {
-      super(null, client);
-    }
-
-    @Override
-    public void setClient(AccumuloClient client) {
-      super.setClient(client);
-    }
-
-    @Override
-    public void setQueuedWork(Map<String,Map<TableId,String>> queuedWork) {
-      super.setQueuedWork(queuedWork);
-    }
-
-    @Override
-    public void setWorkQueue(DistributedWorkQueue workQueue) {
-      super.setWorkQueue(workQueue);
-    }
-
-    @Override
-    public void setMaxQueueSize(int maxQueueSize) {
-      super.setMaxQueueSize(maxQueueSize);
-    }
-
-    @Override
-    public void createWork() {
-      super.createWork();
-    }
-
-    @Override
-    public void setZooCache(ZooCache zooCache) {
-      super.setZooCache(zooCache);
-    }
-
-    @Override
-    public void cleanupFinishedWork() {
-      super.cleanupFinishedWork();
-    }
-
-  }
-
-  @BeforeEach
-  public void init() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    assigner = new MockSequentialWorkAssigner(client);
-    // grant ourselves write to the replication table
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    ReplicationTable.setOnline(client);
-  }
-
-  @Test
-  public void createWorkForFilesInCorrectOrder() throws Exception {
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-    Text serializedTarget = target.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1,
-        file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, target.getSourceTableId(), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, target.getSourceTableId(), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Map<String,Map<TableId,String>> queuedWork = new HashMap<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target), file1);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    assertEquals(1, queuedWork.size());
-    assertTrue(queuedWork.containsKey("cluster1"));
-    Map<TableId,String> cluster1Work = queuedWork.get("cluster1");
-    assertEquals(1, cluster1Work.size());
-    assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
-    assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target),
-        cluster1Work.get(target.getSourceTableId()));
-  }
-
-  @Test
-  public void workAcrossTablesHappensConcurrently() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-    Text serializedTarget1 = target1.toText();
-
-    ReplicationTarget target2 = new ReplicationTarget("cluster1", "table2", TableId.of("2"));
-    Text serializedTarget2 = target2.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1,
-        file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, target1.getSourceTableId(), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, target2.getSourceTableId(), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Map<String,Map<TableId,String>> queuedWork = new HashMap<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1),
-        file1);
-    expectLastCall().once();
-
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2),
-        file2);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    assertEquals(1, queuedWork.size());
-    assertTrue(queuedWork.containsKey("cluster1"));
-
-    Map<TableId,String> cluster1Work = queuedWork.get("cluster1");
-    assertEquals(2, cluster1Work.size());
-    assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
-    assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1),
-        cluster1Work.get(target1.getSourceTableId()));
-
-    assertTrue(cluster1Work.containsKey(target2.getSourceTableId()));
-    assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2),
-        cluster1Work.get(target2.getSourceTableId()));
-  }
-
-  @Test
-  public void workAcrossPeersHappensConcurrently() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-    Text serializedTarget1 = target1.toText();
-
-    ReplicationTarget target2 = new ReplicationTarget("cluster2", "table1", TableId.of("1"));
-    Text serializedTarget2 = target2.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1,
-        file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, target1.getSourceTableId(), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, target2.getSourceTableId(), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Map<String,Map<TableId,String>> queuedWork = new HashMap<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1),
-        file1);
-    expectLastCall().once();
-
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2),
-        file2);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    assertEquals(2, queuedWork.size());
-    assertTrue(queuedWork.containsKey("cluster1"));
-
-    Map<TableId,String> cluster1Work = queuedWork.get("cluster1");
-    assertEquals(1, cluster1Work.size());
-    assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
-    assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1),
-        cluster1Work.get(target1.getSourceTableId()));
-
-    Map<TableId,String> cluster2Work = queuedWork.get("cluster2");
-    assertEquals(1, cluster2Work.size());
-    assertTrue(cluster2Work.containsKey(target2.getSourceTableId()));
-    assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2),
-        cluster2Work.get(target2.getSourceTableId()));
-  }
-
-  @Test
-  public void reprocessingOfCompletedWorkRemovesWork() throws Exception {
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-    Text serializedTarget = target.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1,
-        file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(100).setEnd(100).setClosed(true)
-        .setInfiniteEnd(false).setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false)
-        .setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, target.getSourceTableId(), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, target.getSourceTableId(), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-
-    // Treat filename1 as we have already submitted it for replication
-    Map<String,Map<TableId,String>> queuedWork = new HashMap<>();
-    Map<TableId,String> queuedWorkForCluster = new HashMap<>();
-    queuedWorkForCluster.put(target.getSourceTableId(),
-        DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target));
-    queuedWork.put("cluster1", queuedWorkForCluster);
-
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target), file2);
-    expectLastCall().once();
-
-    // file2 is queued because we remove file1 because it's fully replicated
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    assertEquals(1, queuedWork.size());
-    assertTrue(queuedWork.containsKey("cluster1"));
-    Map<TableId,String> cluster1Work = queuedWork.get("cluster1");
-    assertEquals(1, cluster1Work.size());
-    assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
-    assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target),
-        cluster1Work.get(target.getSourceTableId()));
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java b/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
deleted file mode 100644
index 0b71de2..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/StatusCombinerMacIT.java
+++ /dev/null
@@ -1,144 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.time.Duration;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.cluster.ClusterUser;
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.conf.Property;
-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.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.harness.SharedMiniClusterBase;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class StatusCombinerMacIT extends SharedMiniClusterBase {
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(1);
-  }
-
-  @BeforeAll
-  public static void setup() throws Exception {
-    SharedMiniClusterBase.startMiniCluster();
-  }
-
-  @AfterAll
-  public static void teardown() {
-    SharedMiniClusterBase.stopMiniCluster();
-  }
-
-  @Test
-  public void testCombinerSetOnMetadata() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      TableOperations tops = client.tableOperations();
-      Map<String,EnumSet<IteratorScope>> iterators = tops.listIterators(MetadataTable.NAME);
-
-      assertTrue(iterators.containsKey(ReplicationTableUtil.COMBINER_NAME));
-      EnumSet<IteratorScope> scopes = iterators.get(ReplicationTableUtil.COMBINER_NAME);
-      assertEquals(3, scopes.size());
-      assertTrue(scopes.contains(IteratorScope.scan));
-      assertTrue(scopes.contains(IteratorScope.minc));
-      assertTrue(scopes.contains(IteratorScope.majc));
-
-      Map<String,String> config = tops.getConfiguration(MetadataTable.NAME);
-      Map<String,String> properties = Map.copyOf(config);
-
-      for (IteratorScope scope : scopes) {
-        String key = Property.TABLE_ITERATOR_PREFIX.getKey() + scope.name() + "."
-            + ReplicationTableUtil.COMBINER_NAME + ".opt.columns";
-        assertTrue(properties.containsKey(key), "Properties did not contain key : " + key);
-        assertEquals(ReplicationSection.COLF.toString(), properties.get(key));
-      }
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
-      ClusterUser user = getAdminUser();
-
-      ReplicationTable.setOnline(client);
-      client.securityOperations().grantTablePermission(user.getPrincipal(), ReplicationTable.NAME,
-          TablePermission.WRITE);
-      BatchWriter bw = ReplicationTable.getBatchWriter(client);
-      long createTime = System.currentTimeMillis();
-      try {
-        Mutation m = new Mutation(
-            "file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
-        StatusSection.add(m, TableId.of("1"), StatusUtil.fileCreatedValue(createTime));
-        bw.addMutation(m);
-      } finally {
-        bw.close();
-      }
-
-      Entry<Key,Value> entry;
-      try (Scanner s = ReplicationTable.getScanner(client)) {
-        entry = getOnlyElement(s);
-        assertEquals(StatusUtil.fileCreatedValue(createTime), entry.getValue());
-
-        bw = ReplicationTable.getBatchWriter(client);
-        try {
-          Mutation m = new Mutation(
-              "file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
-          StatusSection.add(m, TableId.of("1"),
-              ProtobufUtil.toValue(StatusUtil.replicated(Long.MAX_VALUE)));
-          bw.addMutation(m);
-        } finally {
-          bw.close();
-        }
-      }
-
-      try (Scanner s = ReplicationTable.getScanner(client)) {
-        entry = getOnlyElement(s);
-        Status stat = Status.parseFrom(entry.getValue().get());
-        assertEquals(Long.MAX_VALUE, stat.getBegin());
-      }
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java
deleted file mode 100644
index e25e0e0..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java
+++ /dev/null
@@ -1,378 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-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.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.StatusMaker;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class StatusMakerIT extends ConfigurableMacBase {
-
-  private AccumuloClient client;
-  private VolumeManager fs;
-
-  @BeforeEach
-  public void setupInstance() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    ReplicationTable.setOnline(client);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-    fs = EasyMock.mock(VolumeManager.class);
-  }
-
-  @Test
-  public void statusRecordsCreated() throws Exception {
-    String sourceTable = testName();
-    client.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(client, sourceTable);
-
-    BatchWriter bw = client.createBatchWriter(sourceTable);
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    Set<String> files =
-        Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-            walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    int index = 1;
-    long timeCreated = 0;
-    Map<String,Long> fileToTimeCreated = new HashMap<>();
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)),
-          StatusUtil.fileCreatedValue(timeCreated));
-      fileToTimeCreated.put(file, timeCreated);
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-      timeCreated++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(client, fs);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      StatusSection.limit(s);
-      Text file = new Text();
-      for (Entry<Key,Value> entry : s) {
-        StatusSection.getFile(entry.getKey(), file);
-        TableId tableId = StatusSection.getTableId(entry.getKey());
-
-        assertTrue(files.contains(file.toString()), "Found unexpected file: " + file);
-        assertEquals(fileToTableId.get(file.toString()), Integer.valueOf(tableId.canonical()));
-        timeCreated = fileToTimeCreated.get(file.toString());
-        assertNotNull(timeCreated);
-        assertEquals(StatusUtil.fileCreated(timeCreated), Status.parseFrom(entry.getValue().get()));
-      }
-    }
-  }
-
-  @Test
-  public void openMessagesAreNotDeleted() throws Exception {
-    String sourceTable = testName();
-    client.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(client, sourceTable);
-
-    BatchWriter bw = client.createBatchWriter(sourceTable);
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    Set<String> files =
-        Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-            walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    int index = 1;
-    long timeCreated = 0;
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)),
-          StatusUtil.fileCreatedValue(timeCreated));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-      timeCreated++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(client, fs);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    try (Scanner s = client.createScanner(sourceTable, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      assertEquals(files.size(), Iterables.size(s));
-    }
-  }
-
-  @Test
-  public void closedMessagesAreDeleted() throws Exception {
-    String sourceTable = testName();
-    client.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(client, sourceTable);
-
-    BatchWriter bw = client.createBatchWriter(sourceTable);
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    Set<String> files =
-        Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-            walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true)
-        .setCreatedTime(System.currentTimeMillis()).build();
-
-    int index = 1;
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(stat));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(client, fs);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    try (Scanner s = client.createScanner(sourceTable, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      for (Entry<Key,Value> e : s) {
-        System.out.println(e.getKey().toStringNoTruncate() + " " + e.getValue());
-      }
-    }
-
-    try (Scanner s = client.createScanner(sourceTable, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      assertEquals(0, Iterables.size(s));
-    }
-  }
-
-  @Test
-  public void closedMessagesCreateOrderRecords() throws Exception {
-    String sourceTable = testName();
-    client.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(client, sourceTable);
-
-    BatchWriter bw = client.createBatchWriter(sourceTable);
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    List<String> files = Arrays.asList(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-        walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    Status.Builder statBuilder =
-        Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true);
-
-    int index = 1;
-    long time = System.currentTimeMillis();
-    for (String file : files) {
-      statBuilder.setCreatedTime(time++);
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)),
-          ProtobufUtil.toValue(statBuilder.build()));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(client, fs);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    Iterator<Entry<Key,Value>> iter;
-    Iterator<String> expectedFiles;
-    try (Scanner s = client.createScanner(sourceTable, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      s.fetchColumnFamily(ReplicationSection.COLF);
-      assertEquals(0, Iterables.size(s));
-    }
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      OrderSection.limit(s);
-      iter = s.iterator();
-      assertTrue(iter.hasNext(), "Found no order records in replication table");
-
-      expectedFiles = files.iterator();
-      Text buff = new Text();
-      while (expectedFiles.hasNext() && iter.hasNext()) {
-        String file = expectedFiles.next();
-        Entry<Key,Value> entry = iter.next();
-
-        assertEquals(file, OrderSection.getFile(entry.getKey(), buff));
-        OrderSection.getTableId(entry.getKey(), buff);
-        assertEquals(fileToTableId.get(file).intValue(), Integer.parseInt(buff.toString()));
-      }
-    }
-    assertFalse(expectedFiles.hasNext(), "Found more files unexpectedly");
-    assertFalse(iter.hasNext(), "Found more entries in replication table unexpectedly");
-  }
-
-  @Test
-  public void orderRecordsCreatedWithNoCreatedTime() throws Exception {
-    String sourceTable = testName();
-    client.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(client, sourceTable);
-
-    BatchWriter bw = client.createBatchWriter(sourceTable);
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    List<String> files = Arrays.asList(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-        walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID());
-    Map<String,Long> fileToTableId = new HashMap<>();
-
-    Status.Builder statBuilder =
-        Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true);
-
-    Map<String,Long> statuses = new HashMap<>();
-    long index = 1;
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Long.toString(index)),
-          ProtobufUtil.toValue(statBuilder.build()));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-
-      FileStatus status = EasyMock.mock(FileStatus.class);
-      EasyMock.expect(status.getModificationTime()).andReturn(index);
-      EasyMock.replay(status);
-      statuses.put(file, index);
-
-      EasyMock.expect(fs.exists(new Path(file))).andReturn(true);
-      EasyMock.expect(fs.getFileStatus(new Path(file))).andReturn(status);
-
-      index++;
-    }
-
-    EasyMock.replay(fs);
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(client, fs);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    Scanner s = client.createScanner(sourceTable, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    s.fetchColumnFamily(ReplicationSection.COLF);
-    assertEquals(0, Iterables.size(s));
-
-    s = ReplicationTable.getScanner(client);
-    OrderSection.limit(s);
-    Iterator<Entry<Key,Value>> iter = s.iterator();
-    assertTrue(iter.hasNext(), "Found no order records in replication table");
-
-    Iterator<String> expectedFiles = files.iterator();
-    Text buff = new Text();
-    while (expectedFiles.hasNext() && iter.hasNext()) {
-      String file = expectedFiles.next();
-      Entry<Key,Value> entry = iter.next();
-
-      assertEquals(file, OrderSection.getFile(entry.getKey(), buff));
-      OrderSection.getTableId(entry.getKey(), buff);
-      assertEquals(fileToTableId.get(file).intValue(), Integer.parseInt(buff.toString()));
-      Status status = Status.parseFrom(entry.getValue().get());
-      assertTrue(status.hasCreatedTime());
-      assertEquals((long) statuses.get(file), status.getCreatedTime());
-    }
-
-    assertFalse(expectedFiles.hasNext(), "Found more files unexpectedly");
-    assertFalse(iter.hasNext(), "Found more entries in replication table unexpectedly");
-
-    s = client.createScanner(sourceTable, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    s.fetchColumnFamily(ReplicationSection.COLF);
-    assertEquals(0, Iterables.size(s));
-
-    s = ReplicationTable.getScanner(client);
-    s.setRange(ReplicationSection.getRange());
-    iter = s.iterator();
-    assertTrue(iter.hasNext(), "Found no stat records in replication table");
-
-    Collections.sort(files);
-    expectedFiles = files.iterator();
-    while (expectedFiles.hasNext() && iter.hasNext()) {
-      String file = expectedFiles.next();
-      Entry<Key,Value> entry = iter.next();
-      Status status = Status.parseFrom(entry.getValue().get());
-      assertTrue(status.hasCreatedTime());
-      assertEquals((long) statuses.get(file), status.getCreatedTime());
-    }
-
-    assertFalse(expectedFiles.hasNext(), "Found more files unexpectedly");
-    assertFalse(iter.hasNext(), "Found more entries in replication table unexpectedly");
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java
deleted file mode 100644
index 73a3f2a..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java
+++ /dev/null
@@ -1,256 +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.test.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.UnorderedWorkAssigner;
-import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class UnorderedWorkAssignerIT extends ConfigurableMacBase {
-
-  private AccumuloClient client;
-  private MockUnorderedWorkAssigner assigner;
-
-  private static class MockUnorderedWorkAssigner extends UnorderedWorkAssigner {
-    public MockUnorderedWorkAssigner(AccumuloClient client) {
-      super(null, client);
-    }
-
-    @Override
-    protected void setQueuedWork(Set<String> queuedWork) {
-      super.setQueuedWork(queuedWork);
-    }
-
-    @Override
-    protected void setWorkQueue(DistributedWorkQueue workQueue) {
-      super.setWorkQueue(workQueue);
-    }
-
-    @Override
-    protected boolean queueWork(Path path, ReplicationTarget target) {
-      return super.queueWork(path, target);
-    }
-
-    @Override
-    protected void initializeQueuedWork() {
-      super.initializeQueuedWork();
-    }
-
-    @Override
-    protected Set<String> getQueuedWork() {
-      return super.getQueuedWork();
-    }
-
-    @Override
-    protected void setClient(AccumuloClient client) {
-      super.setClient(client);
-    }
-
-    @Override
-    protected void setMaxQueueSize(int maxQueueSize) {
-      super.setMaxQueueSize(maxQueueSize);
-    }
-
-    @Override
-    protected void createWork() {
-      super.createWork();
-    }
-
-    @Override
-    protected void setZooCache(ZooCache zooCache) {
-      super.setZooCache(zooCache);
-    }
-
-    @Override
-    protected void cleanupFinishedWork() {
-      super.cleanupFinishedWork();
-    }
-  }
-
-  @BeforeEach
-  public void init() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    assigner = new MockUnorderedWorkAssigner(client);
-    ReplicationTable.setOnline(client);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-  }
-
-  @Test
-  public void createWorkForFilesNeedingIt() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", TableId.of("1")),
-        target2 = new ReplicationTarget("cluster1", "table2", TableId.of("2"));
-    Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
-    String keyTarget1 =
-        target1.getPeerName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
-            + target1.getRemoteIdentifier() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
-            + target1.getSourceTableId(),
-        keyTarget2 = target2.getPeerName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
-            + target2.getRemoteIdentifier() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
-            + target2.getSourceTableId();
-
-    Status.Builder builder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true)
-        .setClosed(false).setCreatedTime(5L);
-    Status status1 = builder.build();
-    builder.setCreatedTime(10L);
-    Status status2 = builder.build();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
-    String file1 = "/accumulo/wal/tserver+port/" + filename1,
-        file2 = "/accumulo/wal/tserver+port/" + filename2;
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(status1));
-    bw.addMutation(m);
-    m = OrderSection.createMutation(file1, status1.getCreatedTime());
-    OrderSection.add(m, target1.getSourceTableId(), ProtobufUtil.toValue(status1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(status2));
-    bw.addMutation(m);
-    m = OrderSection.createMutation(file2, status2.getCreatedTime());
-    OrderSection.add(m, target2.getSourceTableId(), ProtobufUtil.toValue(status2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    HashSet<String> queuedWork = new HashSet<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the order they were created
-    String key = filename1 + "|" + keyTarget1;
-    workQueue.addWork(key, file1);
-    expectLastCall().once();
-
-    key = filename2 + "|" + keyTarget2;
-    workQueue.addWork(key, file2);
-    expectLastCall().once();
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-  }
-
-  @Test
-  public void doNotCreateWorkForFilesNotNeedingIt() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", TableId.of("1")),
-        target2 = new ReplicationTarget("cluster1", "table2", TableId.of("2"));
-    Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
-    String file1 = "/accumulo/wal/tserver+port/" + filename1,
-        file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, StatusUtil.fileCreatedValue(5));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, StatusUtil.fileCreatedValue(10));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    HashSet<String> queuedWork = new HashSet<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-  }
-
-  @Test
-  public void workNotReAdded() throws Exception {
-    Set<String> queuedWork = new HashSet<>();
-
-    assigner.setQueuedWork(queuedWork);
-
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", TableId.of("1"));
-    String serializedTarget = target.getPeerName()
-        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
-        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
-
-    queuedWork.add("wal1|" + serializedTarget);
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    String file1 = "/accumulo/wal/tserver+port/wal1";
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, target.toText(), StatusUtil.openWithUnknownLengthValue());
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
deleted file mode 100644
index fbde6c9..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
+++ /dev/null
@@ -1,810 +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.test.replication;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.PartialKey;
-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.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.UnorderedWorkAssigner;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.miniclusterImpl.ProcessReference;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacBase {
-  private static final Logger log =
-      LoggerFactory.getLogger(UnorderedWorkAssignerReplicationIT.class);
-
-  private ExecutorService executor;
-  private int timeoutFactor = 1;
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(6);
-  }
-
-  @BeforeEach
-  public void createExecutor() {
-    executor = Executors.newSingleThreadExecutor();
-
-    try {
-      timeoutFactor = Integer.parseInt(System.getProperty("timeout.factor"));
-    } catch (NumberFormatException exception) {
-      log.warn("Could not parse timeout.factor, not increasing timeout.");
-    }
-
-    assertTrue(timeoutFactor > 0, "The timeout factor must be a positive, non-zero value");
-  }
-
-  @AfterEach
-  public void stopExecutor() {
-    if (executor != null) {
-      executor.shutdownNow();
-    }
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
-    cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
-    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.TSERV_WAL_MAX_SIZE, "2M");
-    cfg.setProperty(Property.GC_CYCLE_START, "1s");
-    cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
-    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-    cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-    cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
-    cfg.setProperty(Property.REPLICATION_NAME, "manager");
-    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, UnorderedWorkAssigner.class.getName());
-    cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
-    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  /**
-   * Use the same SSL and credential provider configuration that is set up by AbstractMacIT for the
-   * other MAC used for replication
-   */
-  private void updatePeerConfigFromPrimary(MiniAccumuloConfigImpl primaryCfg,
-      MiniAccumuloConfigImpl peerCfg) {
-    // Set the same SSL information from the primary when present
-    Map<String,String> primarySiteConfig = primaryCfg.getSiteConfig();
-    if ("true".equals(primarySiteConfig.get(Property.INSTANCE_RPC_SSL_ENABLED.getKey()))) {
-      Map<String,String> peerSiteConfig = new HashMap<>();
-      peerSiteConfig.put(Property.INSTANCE_RPC_SSL_ENABLED.getKey(), "true");
-      String keystorePath = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey());
-      assertNotNull(keystorePath, "Keystore Path was null");
-      peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PATH.getKey(), keystorePath);
-      String truststorePath = primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey());
-      assertNotNull(truststorePath, "Truststore Path was null");
-      peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PATH.getKey(), truststorePath);
-
-      // Passwords might be stored in CredentialProvider
-      String keystorePassword = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey());
-      if (keystorePassword != null) {
-        peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey(), keystorePassword);
-      }
-      String truststorePassword =
-          primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey());
-      if (truststorePassword != null) {
-        peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
-      }
-
-      System.out.println("Setting site configuration for peer " + peerSiteConfig);
-      peerCfg.setSiteConfig(peerSiteConfig);
-    }
-
-    // Use the CredentialProvider if the primary also uses one
-    String credProvider =
-        primarySiteConfig.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
-    if (credProvider != null) {
-      Map<String,String> peerSiteConfig = peerCfg.getSiteConfig();
-      peerSiteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(),
-          credProvider);
-      peerCfg.setSiteConfig(peerSiteConfig);
-    }
-  }
-
-  @Test
-  public void dataWasReplicatedToThePeer() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-    MiniAccumuloClusterImpl peerCluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peerCluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      ReplicationTable.setOnline(clientManager);
-
-      String peerUserName = "peer", peerPassword = "foo";
-
-      String peerClusterName = "peer";
-
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
-                  peerCluster.getZooKeepers())));
-
-      final String managerTable = "manager", peerTable = "peer";
-
-      clientPeer.tableOperations().create(peerTable);
-      String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
-      assertNotNull(peerTableId);
-
-      Map<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
-
-      clientManager.tableOperations().create(managerTable,
-          new NewTableConfiguration().setProperties(props));
-      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
-      assertNotNull(managerTableId);
-
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable,
-          TablePermission.WRITE);
-
-      // Wait for zookeeper updates (configuration) to propagate
-      sleepUninterruptibly(3, TimeUnit.SECONDS);
-
-      // Write some data to table1
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
-        for (int rows = 0; rows < 5000; rows++) {
-          Mutation m = new Mutation(Integer.toString(rows));
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      final Set<String> filesNeedingReplication =
-          clientManager.replicationOperations().referencedFiles(managerTable);
-
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-      cluster.exec(TabletServer.class);
-
-      log.info("TabletServer restarted");
-      try (Scanner scanner = ReplicationTable.getScanner(clientManager)) {
-        scanner.forEach((k, v) -> {});
-      }
-      log.info("TabletServer is online");
-
-      log.info("");
-      log.info("Fetching metadata records:");
-      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> kv : scanner) {
-          if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-          } else {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(), kv.getValue());
-          }
-        }
-      }
-
-      log.info("");
-      log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientManager)) {
-        for (Entry<Key,Value> kv : scanner) {
-          log.info("{} {}", kv.getKey().toStringNoTruncate(),
-              ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-        }
-      }
-
-      Future<Boolean> future = executor.submit(() -> {
-        clientManager.replicationOperations().drain(managerTable, filesNeedingReplication);
-        log.info("Drain completed");
-        return true;
-      });
-
-      long timeoutSeconds = timeoutFactor * 30L;
-      try {
-        future.get(timeoutSeconds, TimeUnit.SECONDS);
-      } catch (TimeoutException e) {
-        future.cancel(true);
-        fail("Drain did not finish within " + timeoutSeconds + " seconds");
-      }
-
-      log.info("drain completed");
-
-      log.info("");
-      log.info("Fetching metadata records:");
-      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> kv : scanner) {
-          if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(),
-                ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-          } else {
-            log.info("{} {}", kv.getKey().toStringNoTruncate(), kv.getValue());
-          }
-        }
-      }
-
-      log.info("");
-      log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientManager)) {
-        for (Entry<Key,Value> kv : scanner) {
-          log.info("{} {}", kv.getKey().toStringNoTruncate(),
-              ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-        }
-      }
-
-      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
-          Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
-        Entry<Key,Value> managerEntry = null, peerEntry = null;
-        while (managerIter.hasNext() && peerIter.hasNext()) {
-          managerEntry = managerIter.next();
-          peerEntry = peerIter.next();
-          assertEquals(0,
-              managerEntry.getKey().compareTo(peerEntry.getKey(),
-                  PartialKey.ROW_COLFAM_COLQUAL_COLVIS),
-              managerEntry.getKey() + " was not equal to " + peerEntry.getKey());
-          assertEquals(managerEntry.getValue(), peerEntry.getValue());
-        }
-
-        log.info("Last manager entry: {}", managerEntry);
-        log.info("Last peer entry: {}", peerEntry);
-
-        assertFalse(managerIter.hasNext(), "Had more data to read from the manager");
-        assertFalse(peerIter.hasNext(), "Had more data to read from the peer");
-      }
-    } finally {
-      peerCluster.stop();
-    }
-  }
-
-  @Test
-  public void dataReplicatedToCorrectTable() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-    MiniAccumuloClusterImpl peer1Cluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peer1Cluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      String peerClusterName = "peer";
-      String peerUserName = "peer", peerPassword = "foo";
-
-      // Create local user
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
-                  peer1Cluster.getZooKeepers())));
-
-      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
-          peerTable2 = "peer2";
-
-      // Create tables
-
-      clientPeer.tableOperations().create(peerTable1);
-      String peerTableId1 = clientPeer.tableOperations().tableIdMap().get(peerTable1);
-      assertNotNull(peerTableId1);
-
-      clientPeer.tableOperations().create(peerTable2);
-      String peerTableId2 = clientPeer.tableOperations().tableIdMap().get(peerTable2);
-      assertNotNull(peerTableId2);
-
-      // Grant write permission
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
-          TablePermission.WRITE);
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable2,
-          TablePermission.WRITE);
-
-      Map<String,String> props1 = new HashMap<>();
-      props1.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
-
-      clientManager.tableOperations().create(managerTable1,
-          new NewTableConfiguration().setProperties(props1));
-      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
-      assertNotNull(managerTableId1);
-
-      Map<String,String> props2 = new HashMap<>();
-      props2.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
-
-      clientManager.tableOperations().create(managerTable2,
-          new NewTableConfiguration().setProperties(props2));
-      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
-      assertNotNull(managerTableId2);
-
-      // Wait for zookeeper updates (configuration) to propagate
-      sleepUninterruptibly(3, TimeUnit.SECONDS);
-
-      // Write some data to table1
-      long managerTable1Records = 0L;
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable1 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-            managerTable1Records++;
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      // Write some data to table2
-      long managerTable2Records = 0L;
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable2 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-            managerTable2Records++;
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      Set<String> filesFor1 = clientManager.replicationOperations().referencedFiles(managerTable1),
-          filesFor2 = clientManager.replicationOperations().referencedFiles(managerTable2);
-
-      while (!ReplicationTable.isOnline(clientManager)) {
-        Thread.sleep(500);
-      }
-
-      // Restart the tserver to force a close on the WAL
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-      cluster.exec(TabletServer.class);
-
-      log.info("Restarted the tserver");
-
-      // Read the data -- the tserver is back up and running
-      try (Scanner scanner = clientManager.createScanner(managerTable1, Authorizations.EMPTY)) {
-        scanner.forEach((k, v) -> {});
-      }
-
-      // Wait for both tables to be replicated
-      log.info("Waiting for {} for {}", filesFor1, managerTable1);
-      clientManager.replicationOperations().drain(managerTable1, filesFor1);
-
-      log.info("Waiting for {} for {}", filesFor2, managerTable2);
-      clientManager.replicationOperations().drain(managerTable2, filesFor2);
-
-      long countTable = 0L;
-      for (int i = 0; i < 5; i++) {
-        countTable = 0L;
-        for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable1),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-
-        log.info("Found {} records in {}", countTable, peerTable1);
-
-        if (managerTable1Records != countTable) {
-          log.warn("Did not find {} expected records in {}, only found {}", managerTable1Records,
-              peerTable1, countTable);
-        }
-      }
-
-      assertEquals(managerTable1Records, countTable);
-
-      for (int i = 0; i < 5; i++) {
-        countTable = 0L;
-        for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable2),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-
-        log.info("Found {} records in {}", countTable, peerTable2);
-
-        if (managerTable2Records != countTable) {
-          log.warn("Did not find {} expected records in {}, only found {}", managerTable2Records,
-              peerTable2, countTable);
-        }
-      }
-
-      assertEquals(managerTable2Records, countTable);
-
-    } finally {
-      peer1Cluster.stop();
-    }
-  }
-
-  @Test
-  public void dataWasReplicatedToThePeerWithoutDrain() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-    MiniAccumuloClusterImpl peerCluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peerCluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      String peerUserName = "repl";
-      String peerPassword = "passwd";
-
-      // Create a user on the peer for replication to use
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      String peerClusterName = "peer";
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
-                  peerCluster.getZooKeepers())));
-
-      // Configure the credentials we should use to authenticate ourselves to the peer for
-      // replication
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      String managerTable = "manager", peerTable = "peer";
-
-      clientPeer.tableOperations().create(peerTable);
-      String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
-      assertNotNull(peerTableId);
-
-      Map<String,String> props = new HashMap<>();
-      props.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
-
-      clientManager.tableOperations().create(managerTable,
-          new NewTableConfiguration().setProperties(props));
-      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
-      assertNotNull(managerTableId);
-
-      // Give our replication user the ability to write to the table
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable,
-          TablePermission.WRITE);
-
-      // Write some data to table1
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
-        for (int rows = 0; rows < 5000; rows++) {
-          Mutation m = new Mutation(Integer.toString(rows));
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      Set<String> files = clientManager.replicationOperations().referencedFiles(managerTable);
-      for (String s : files) {
-        log.info("Found referenced file for {}: {}", managerTable, s);
-      }
-
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-
-      cluster.exec(TabletServer.class);
-
-      try (Scanner scanner = clientManager.createScanner(managerTable, Authorizations.EMPTY)) {
-        scanner.forEach((k, v) -> {});
-      }
-
-      try (var scanner = clientManager.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> kv : scanner) {
-          log.debug("{} {}", kv.getKey().toStringNoTruncate(),
-              ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-        }
-      }
-
-      clientManager.replicationOperations().drain(managerTable, files);
-
-      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
-          Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
-        assertTrue(managerIter.hasNext(), "No data in manager table");
-        assertTrue(peerIter.hasNext(), "No data in peer table");
-        while (managerIter.hasNext() && peerIter.hasNext()) {
-          Entry<Key,Value> managerEntry = managerIter.next(), peerEntry = peerIter.next();
-          assertEquals(0,
-              managerEntry.getKey().compareTo(peerEntry.getKey(),
-                  PartialKey.ROW_COLFAM_COLQUAL_COLVIS),
-              peerEntry.getKey() + " was not equal to " + peerEntry.getKey());
-          assertEquals(managerEntry.getValue(), peerEntry.getValue());
-        }
-
-        assertFalse(managerIter.hasNext(), "Had more data to read from the manager");
-        assertFalse(peerIter.hasNext(), "Had more data to read from the peer");
-      }
-      peerCluster.stop();
-    }
-  }
-
-  @Test
-  public void dataReplicatedToCorrectTableWithoutDrain() throws Exception {
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(
-        createTestDir(this.getClass().getName() + "_" + this.testName() + "_peer"), ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-    MiniAccumuloClusterImpl peer1Cluster = new MiniAccumuloClusterImpl(peerCfg);
-
-    peer1Cluster.start();
-
-    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
-        AccumuloClient clientPeer =
-            peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
-
-      String peerClusterName = "peer";
-
-      String peerUserName = "repl";
-      String peerPassword = "passwd";
-
-      // Create a user on the peer for replication to use
-      clientPeer.securityOperations().createLocalUser(peerUserName,
-          new PasswordToken(peerPassword));
-
-      // Configure the credentials we should use to authenticate ourselves to the peer for
-      // replication
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientManager.instanceOperations()
-          .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-
-      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientManager.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + peerClusterName,
-          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
-                  peer1Cluster.getZooKeepers())));
-
-      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
-          peerTable2 = "peer2";
-      clientPeer.tableOperations().create(peerTable1, new NewTableConfiguration());
-      String peerTableId1 = clientPeer.tableOperations().tableIdMap().get(peerTable1);
-      assertNotNull(peerTableId1);
-
-      clientPeer.tableOperations().create(peerTable2, new NewTableConfiguration());
-      String peerTableId2 = clientPeer.tableOperations().tableIdMap().get(peerTable2);
-      assertNotNull(peerTableId2);
-
-      Map<String,String> props1 = new HashMap<>();
-      props1.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
-
-      clientManager.tableOperations().create(managerTable1,
-          new NewTableConfiguration().setProperties(props1));
-      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
-      assertNotNull(managerTableId1);
-
-      Map<String,String> props2 = new HashMap<>();
-      props2.put(Property.TABLE_REPLICATION.getKey(), "true");
-      props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
-
-      clientManager.tableOperations().create(managerTable2,
-          new NewTableConfiguration().setProperties(props2));
-      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
-      assertNotNull(managerTableId2);
-
-      // Give our replication user the ability to write to the tables
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
-          TablePermission.WRITE);
-      clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable2,
-          TablePermission.WRITE);
-
-      // Wait for zookeeper updates (configuration) to propagate
-      sleepUninterruptibly(3, TimeUnit.SECONDS);
-
-      // Write some data to table1
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable1 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      // Write some data to table2
-      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
-        for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(managerTable2 + rows);
-          for (int cols = 0; cols < 100; cols++) {
-            String value = Integer.toString(cols);
-            m.put(value, "", value);
-          }
-          bw.addMutation(m);
-        }
-      }
-
-      log.info("Wrote all data to manager cluster");
-
-      while (!ReplicationTable.isOnline(clientManager)) {
-        Thread.sleep(500);
-      }
-
-      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        cluster.killProcess(ServerType.TABLET_SERVER, proc);
-      }
-
-      cluster.exec(TabletServer.class);
-
-      // Wait until we fully replicated something
-      boolean fullyReplicated = false;
-      for (int i = 0; i < 10 && !fullyReplicated; i++) {
-        sleepUninterruptibly(timeoutFactor * 2L, TimeUnit.SECONDS);
-
-        try (Scanner s = ReplicationTable.getScanner(clientManager)) {
-          WorkSection.limit(s);
-          for (Entry<Key,Value> entry : s) {
-            Status status = Status.parseFrom(entry.getValue().get());
-            if (StatusUtil.isFullyReplicated(status)) {
-              fullyReplicated |= true;
-            }
-          }
-        }
-      }
-
-      assertNotEquals(0, fullyReplicated);
-
-      long countTable = 0L;
-
-      // Check a few times
-      for (int i = 0; i < 10; i++) {
-        countTable = 0L;
-        for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable1),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-        log.info("Found {} records in {}", countTable, peerTable1);
-        if (countTable > 0) {
-          break;
-        }
-        Thread.sleep(2000);
-      }
-
-      assertTrue(countTable > 0, "Did not find any records in " + peerTable1 + " on peer");
-
-      for (int i = 0; i < 10; i++) {
-        countTable = 0L;
-        for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
-          countTable++;
-          assertTrue(entry.getKey().getRow().toString().startsWith(managerTable2),
-              "Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-                  + entry.getValue());
-        }
-
-        log.info("Found {} records in {}", countTable, peerTable2);
-        if (countTable > 0) {
-          break;
-        }
-        Thread.sleep(2000);
-      }
-      assertTrue(countTable > 0, "Did not find any records in " + peerTable2 + " on peer");
-
-    } finally {
-      peer1Cluster.stop();
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/UnusedWalDoesntCloseReplicationStatusIT.java b/test/src/main/java/org/apache/accumulo/test/replication/UnusedWalDoesntCloseReplicationStatusIT.java
deleted file mode 100644
index 6408ed8..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/UnusedWalDoesntCloseReplicationStatusIT.java
+++ /dev/null
@@ -1,244 +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.test.replication;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.tserver.logger.LogEvents.OPEN;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map.Entry;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-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.accumulo.core.dataImpl.KeyExtent;
-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.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.server.data.ServerMutation;
-import org.apache.accumulo.server.replication.ReplicaSystemFactory;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.accumulo.tserver.log.DfsLogger;
-import org.apache.accumulo.tserver.logger.LogEvents;
-import org.apache.accumulo.tserver.logger.LogFileKey;
-import org.apache.accumulo.tserver.logger.LogFileValue;
-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.io.Text;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Iterables;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class UnusedWalDoesntCloseReplicationStatusIT extends ConfigurableMacBase {
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
-    cfg.setNumTservers(1);
-  }
-
-  @Test
-  public void test() throws Exception {
-    File accumuloDir = this.getCluster().getConfig().getAccumuloDir();
-    final AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
-    final String tableName = getUniqueNames(1)[0];
-
-    client.securityOperations().grantTablePermission("root", MetadataTable.NAME,
-        TablePermission.WRITE);
-    client.tableOperations().create(tableName);
-
-    final TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
-    final int numericTableId = Integer.parseInt(tableId.canonical());
-    final int fakeTableId = numericTableId + 1;
-
-    assertNotNull(tableId, "Did not find table ID");
-
-    client.tableOperations().setProperty(tableName, Property.TABLE_REPLICATION.getKey(), "true");
-    client.tableOperations().setProperty(tableName,
-        Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
-    // just sleep
-    client.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-        ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "50000"));
-
-    FileSystem fs = FileSystem.getLocal(new Configuration());
-    File tserverWalDir =
-        new File(accumuloDir, Constants.WAL_DIR + Path.SEPARATOR + "faketserver+port");
-    File tserverWal = new File(tserverWalDir, UUID.randomUUID().toString());
-    fs.mkdirs(new Path(tserverWalDir.getAbsolutePath()));
-
-    // Make a fake WAL with no data in it for our real table
-    FSDataOutputStream out = fs.create(new Path(tserverWal.getAbsolutePath()));
-
-    out.write(DfsLogger.LOG_FILE_HEADER_V4.getBytes(UTF_8));
-
-    DataOutputStream dos = new DataOutputStream(out);
-    dos.writeUTF("NullCryptoModule");
-
-    // Fake a single update WAL that has a mutation for another table
-    LogFileKey key = new LogFileKey();
-    LogFileValue value = new LogFileValue();
-
-    key.event = OPEN;
-    key.tserverSession = tserverWal.getAbsolutePath();
-    key.filename = tserverWal.getAbsolutePath();
-    key.write(out);
-    value.write(out);
-
-    key.event = LogEvents.DEFINE_TABLET;
-    key.tablet = new KeyExtent(TableId.of(Integer.toString(fakeTableId)), null, null);
-    key.seq = 1L;
-    key.tabletId = 1;
-
-    key.write(dos);
-    value.write(dos);
-
-    key.tablet = null;
-    key.event = LogEvents.MUTATION;
-    key.filename = tserverWal.getAbsolutePath();
-    value.mutations = Arrays.asList(new ServerMutation(new Text("row")));
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.COMPACTION_START;
-    key.filename =
-        accumuloDir.getAbsolutePath() + "/tables/" + fakeTableId + "/t-000001/A000001.rf";
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    key.event = LogEvents.COMPACTION_FINISH;
-    value.mutations = Collections.emptyList();
-
-    key.write(dos);
-    value.write(dos);
-
-    dos.close();
-
-    try (BatchWriter bw = client.createBatchWriter(tableName)) {
-      Mutation m = new Mutation("m");
-      m.put("m", "m", "M");
-      bw.addMutation(m);
-    }
-
-    log.info("State of metadata table after inserting a record");
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-
-      s.setRange(TabletsSection.getRange(tableId));
-      for (Entry<Key,Value> entry : s) {
-        System.out.println(entry.getKey().toStringNoTruncate() + " " + entry.getValue());
-      }
-    }
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      for (Entry<Key,Value> entry : s) {
-        System.out.println(entry.getKey().toStringNoTruncate() + " "
-            + ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
-      }
-
-      log.info("Offline'ing table");
-
-      client.tableOperations().offline(tableName, true);
-
-      // Add our fake WAL to the log column for this table
-      String walUri = tserverWal.toURI().toString();
-      KeyExtent extent = new KeyExtent(tableId, null, null);
-      try (BatchWriter bw = client.createBatchWriter(MetadataTable.NAME)) {
-        Mutation m = new Mutation(extent.toMetaRow());
-        m.put(LogColumnFamily.NAME, new Text("localhost:12345/" + walUri),
-            new Value(walUri + "|1"));
-        bw.addMutation(m);
-
-        // Add a replication entry for our fake WAL
-        m = new Mutation(ReplicationSection.getRowPrefix() + new Path(walUri));
-        m.put(ReplicationSection.COLF, new Text(tableId.canonical()),
-            new Value(StatusUtil.fileCreated(System.currentTimeMillis()).toByteArray()));
-        bw.addMutation(m);
-      }
-
-      log.info("State of metadata after injecting WAL manually");
-    }
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.setRange(TabletsSection.getRange(tableId));
-      for (Entry<Key,Value> entry : s) {
-        log.info("{} {}", entry.getKey().toStringNoTruncate(), entry.getValue());
-      }
-    }
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      for (Entry<Key,Value> entry : s) {
-        log.info("{} {}", entry.getKey().toStringNoTruncate(),
-            ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
-      }
-
-      log.info("Bringing table online");
-      client.tableOperations().online(tableName, true);
-
-      assertEquals(1, Iterables.size(client.createScanner(tableName, Authorizations.EMPTY)));
-
-      log.info("Table has performed recovery, state of metadata:");
-    }
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.setRange(TabletsSection.getRange(tableId));
-      for (Entry<Key,Value> entry : s) {
-        log.info("{} {}", entry.getKey().toStringNoTruncate(), entry.getValue());
-      }
-    }
-
-    try (Scanner s = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      s.setRange(ReplicationSection.getRange());
-      for (Entry<Key,Value> entry : s) {
-        Status status = Status.parseFrom(entry.getValue().get());
-        log.info("{} {}", entry.getKey().toStringNoTruncate(), ProtobufUtil.toString(status));
-        assertFalse(status.getClosed(), "Status record was closed and it should not be");
-      }
-    }
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java
deleted file mode 100644
index 7feef60..0000000
--- a/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java
+++ /dev/null
@@ -1,236 +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.test.replication;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-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.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.replication.WorkMaker;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-import com.google.common.collect.Iterables;
-
-@Disabled("Replication ITs are not stable and not currently maintained")
-@Deprecated
-public class WorkMakerIT extends ConfigurableMacBase {
-
-  private AccumuloClient client;
-
-  private static class MockWorkMaker extends WorkMaker {
-
-    public MockWorkMaker(AccumuloClient client) {
-      super(null, client);
-    }
-
-    @Override
-    public void setBatchWriter(BatchWriter bw) {
-      super.setBatchWriter(bw);
-    }
-
-    @Override
-    public void addWorkRecord(Text file, Value v, Map<String,String> targets,
-        TableId sourceTableId) {
-      super.addWorkRecord(file, v, targets, sourceTableId);
-    }
-
-    @Override
-    public boolean shouldCreateWork(Status status) {
-      return super.shouldCreateWork(status);
-    }
-
-  }
-
-  @BeforeEach
-  public void setupInstance() throws Exception {
-    client = Accumulo.newClient().from(getClientProperties()).build();
-    ReplicationTable.setOnline(client);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.WRITE);
-    client.securityOperations().grantTablePermission(client.whoami(), ReplicationTable.NAME,
-        TablePermission.READ);
-  }
-
-  @Test
-  public void singleUnitSingleTarget() throws Exception {
-    String table = testName();
-    client.tableOperations().create(table);
-    TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(table));
-    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
-
-    // Create a status record for a file
-    long timeCreated = System.currentTimeMillis();
-    Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId.canonical()),
-        StatusUtil.fileCreatedValue(timeCreated));
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    bw.addMutation(m);
-    bw.flush();
-
-    // Assert that we have one record in the status section
-    ReplicationTarget expected;
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      StatusSection.limit(s);
-      assertEquals(1, Iterables.size(s));
-
-      MockWorkMaker workMaker = new MockWorkMaker(client);
-
-      // Invoke the addWorkRecord method to create a Work record from the Status record earlier
-      expected = new ReplicationTarget("remote_cluster_1", "4", tableId);
-      workMaker.setBatchWriter(bw);
-      workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(timeCreated),
-          Map.of("remote_cluster_1", "4"), tableId);
-    }
-
-    // Scan over just the WorkSection
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      WorkSection.limit(s);
-
-      Entry<Key,Value> workEntry = getOnlyElement(s);
-      Key workKey = workEntry.getKey();
-      ReplicationTarget actual = ReplicationTarget.from(workKey.getColumnQualifier());
-
-      assertEquals(file, workKey.getRow().toString());
-      assertEquals(WorkSection.NAME, workKey.getColumnFamily());
-      assertEquals(expected, actual);
-      assertEquals(workEntry.getValue(), StatusUtil.fileCreatedValue(timeCreated));
-    }
-  }
-
-  @Test
-  public void singleUnitMultipleTargets() throws Exception {
-    String table = testName();
-    client.tableOperations().create(table);
-
-    TableId tableId = TableId.of(client.tableOperations().tableIdMap().get(table));
-
-    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
-
-    Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId.canonical()),
-        StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    bw.addMutation(m);
-    bw.flush();
-
-    // Assert that we have one record in the status section
-    Set<ReplicationTarget> expectedTargets = new HashSet<>();
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      StatusSection.limit(s);
-      assertEquals(1, Iterables.size(s));
-
-      MockWorkMaker workMaker = new MockWorkMaker(client);
-
-      Map<String,String> targetClusters =
-          Map.of("remote_cluster_1", "4", "remote_cluster_2", "6", "remote_cluster_3", "8");
-
-      for (Entry<String,String> cluster : targetClusters.entrySet()) {
-        expectedTargets.add(new ReplicationTarget(cluster.getKey(), cluster.getValue(), tableId));
-      }
-      workMaker.setBatchWriter(bw);
-      workMaker.addWorkRecord(new Text(file),
-          StatusUtil.fileCreatedValue(System.currentTimeMillis()), targetClusters, tableId);
-    }
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      WorkSection.limit(s);
-
-      Set<ReplicationTarget> actualTargets = new HashSet<>();
-      for (Entry<Key,Value> entry : s) {
-        assertEquals(file, entry.getKey().getRow().toString());
-        assertEquals(WorkSection.NAME, entry.getKey().getColumnFamily());
-
-        ReplicationTarget target = ReplicationTarget.from(entry.getKey().getColumnQualifier());
-        actualTargets.add(target);
-      }
-
-      for (ReplicationTarget expected : expectedTargets) {
-        assertTrue(actualTargets.contains(expected), "Did not find expected target: " + expected);
-        actualTargets.remove(expected);
-      }
-
-      assertTrue(actualTargets.isEmpty(), "Found extra replication work entries: " + actualTargets);
-    }
-  }
-
-  @Test
-  public void dontCreateWorkForEntriesWithNothingToReplicate() throws Exception {
-    String table = testName();
-    client.tableOperations().create(table);
-    String tableId = client.tableOperations().tableIdMap().get(table);
-    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
-
-    Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId),
-        StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    BatchWriter bw = ReplicationTable.getBatchWriter(client);
-    bw.addMutation(m);
-    bw.flush();
-
-    // Assert that we have one record in the status section
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      StatusSection.limit(s);
-      assertEquals(1, Iterables.size(s));
-
-      MockWorkMaker workMaker = new MockWorkMaker(client);
-
-      client.tableOperations().setProperty(ReplicationTable.NAME,
-          Property.TABLE_REPLICATION_TARGET.getKey() + "remote_cluster_1", "4");
-
-      workMaker.setBatchWriter(bw);
-
-      // If we don't shortcircuit out, we should get an exception because
-      // ServerConfiguration.getTableConfiguration
-      // won't work with MockAccumulo
-      workMaker.run();
-    }
-
-    try (Scanner s = ReplicationTable.getScanner(client)) {
-      WorkSection.limit(s);
-
-      assertEquals(0, Iterables.size(s));
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
index 7ac2a5a..e9164c7 100644
--- a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
@@ -32,7 +32,7 @@
 import org.apache.accumulo.core.clientImpl.Credentials;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.security.SystemCredentials;
@@ -82,7 +82,8 @@
       try (AccumuloClient client = Accumulo.newClient().from(context.getProperties())
           .as(creds.getPrincipal(), creds.getToken()).build()) {
         client.securityOperations().authenticateUser(creds.getPrincipal(), creds.getToken());
-        try (Scanner scan = client.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
+        try (Scanner scan =
+            client.createScanner(AccumuloTable.ROOT.tableName(), Authorizations.EMPTY)) {
           scan.forEach((k, v) -> {});
         } catch (RuntimeException e) {
           e.printStackTrace(System.err);
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ConfigSetIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ConfigSetIT.java
index 9da4c6b..71a5e92 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ConfigSetIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ConfigSetIT.java
@@ -49,6 +49,7 @@
   private static final Logger log = LoggerFactory.getLogger(ConfigSetIT.class);
 
   @Test
+  @SuppressWarnings("deprecation")
   public void setInvalidJson() throws Exception {
     log.debug("Starting setInvalidJson test ------------------");
 
@@ -61,7 +62,6 @@
 
     try (AccumuloClient client =
         getCluster().createAccumuloClient("root", new PasswordToken(getRootPassword()))) {
-
       client.instanceOperations().setProperty(TSERV_COMPACTION_SERVICE_ROOT_EXECUTORS.getKey(),
           validJson);
       assertThrows(AccumuloException.class, () -> client.instanceOperations()
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateNamespaceIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateNamespaceIT.java
index 1878308..4dbe845 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateNamespaceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateNamespaceIT.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.test.shell;
 
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -40,7 +39,6 @@
 import org.junit.jupiter.api.Test;
 
 @Tag(MINI_CLUSTER_ONLY)
-@Tag(SUNNY_DAY)
 public class ShellCreateNamespaceIT extends SharedMiniClusterBase {
 
   private MockShell ts;
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java
index 18309fe..a72efbd 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellCreateTableIT.java
@@ -21,8 +21,8 @@
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.nio.file.Files.newBufferedReader;
 import static java.util.Objects.requireNonNull;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -67,7 +67,6 @@
 import org.junit.jupiter.api.Test;
 
 @Tag(MINI_CLUSTER_ONLY)
-@Tag(SUNNY_DAY)
 public class ShellCreateTableIT extends SharedMiniClusterBase {
 
   private MockShell ts;
@@ -884,7 +883,7 @@
     Set<Text> splits = new HashSet<>();
     for (int i = 0; i < numItems; i++) {
       byte[] split = new byte[len];
-      random.nextBytes(split);
+      RANDOM.get().nextBytes(split);
       splits.add(new Text(split));
     }
     return splits;
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellIT.java
index 650ebac..48b2bd7 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellIT.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.test.shell;
 
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -28,11 +27,8 @@
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.file.Files;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
 import java.time.Duration;
 import java.util.Arrays;
-import java.util.Date;
 import java.util.List;
 import java.util.Map;
 import java.util.TimeZone;
@@ -56,7 +52,6 @@
 import org.slf4j.LoggerFactory;
 
 @Tag(MINI_CLUSTER_ONLY)
-@Tag(SUNNY_DAY)
 public class ShellIT extends SharedMiniClusterBase {
 
   @Override
@@ -340,10 +335,7 @@
 
     // delete will show the timestamp
     exec("deletemany -r 1 -f -st", true, "[DELETED] 1 1:1 [] 1");
-
-    // DeleteManyCommand has its own Formatter (DeleterFormatter), so it does not honor the -fm flag
-    exec("deletemany -r 2 -f -st -fm org.apache.accumulo.core.util.format.DateStringFormatter",
-        true, "[DELETED] 2 2:2 [] 2");
+    exec("deletemany -r 2 -f -st", true, "[DELETED] 2 2:2 [] 2");
 
     exec("setauths -c ", true);
     exec("deletetable test -f", true, "Table: [test] has been deleted");
@@ -424,36 +416,11 @@
     String expectedFew = "1 123:12345 [12345678] 123456789\t12345";
     exec("scan -st", true, expected);
     exec("scan -st -f 5", true, expectedFew);
-    // also prove that BinaryFormatter behaves same as the default
-    exec("scan -st -fm org.apache.accumulo.core.util.format.BinaryFormatter", true, expected);
-    exec("scan -st -f 5 -fm org.apache.accumulo.core.util.format.BinaryFormatter", true,
-        expectedFew);
     exec("setauths -c", true);
     exec("deletetable " + name + " -f", true, "Table: [" + name + "] has been deleted");
   }
 
   @Test
-  public void scanDateStringFormatterTest() throws IOException {
-    Shell.log.debug("Starting scan dateStringFormatter test --------------------------");
-    exec("createtable t", true);
-    exec("insert r f q v -ts 0", true);
-    @SuppressWarnings("deprecation")
-    DateFormat dateFormat =
-        new SimpleDateFormat(org.apache.accumulo.core.util.format.DateStringFormatter.DATE_FORMAT);
-    String expected = String.format("r f:q [] %s\tv", dateFormat.format(new Date(0)));
-    // historically, showing few did not pertain to ColVis or Timestamp
-    String expectedNoTimestamp = "r f:q []\tv";
-    exec("scan -fm org.apache.accumulo.core.util.format.DateStringFormatter -st", true, expected);
-    exec("scan -fm org.apache.accumulo.core.util.format.DateStringFormatter -st -f 1000", true,
-        expected);
-    exec("scan -fm org.apache.accumulo.core.util.format.DateStringFormatter -st -f 5", true,
-        expected);
-    exec("scan -fm org.apache.accumulo.core.util.format.DateStringFormatter", true,
-        expectedNoTimestamp);
-    exec("deletetable t -f", true, "Table: [t] has been deleted");
-  }
-
-  @Test
   public void grepTest() throws IOException {
     Shell.log.debug("Starting grep test --------------------------");
     exec("grep", false, "java.lang.IllegalStateException: Not in a table context");
diff --git a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
index 59805e5..9433b0b 100644
--- a/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/shell/ShellServerIT.java
@@ -19,9 +19,9 @@
 package org.apache.accumulo.test.shell;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
-import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -36,7 +36,6 @@
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -46,7 +45,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
 import org.apache.accumulo.core.Constants;
@@ -71,7 +69,8 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.spi.crypto.NoCryptoServiceFactory;
@@ -80,7 +79,6 @@
 import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.compaction.TestCompactionStrategy;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.accumulo.test.util.Wait;
 import org.apache.hadoop.conf.Configuration;
@@ -103,21 +101,14 @@
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 @Tag(MINI_CLUSTER_ONLY)
-@Tag(SUNNY_DAY)
 public class ShellServerIT extends SharedMiniClusterBase {
 
-  @SuppressWarnings("removal")
-  private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
-      Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-
   private static final Logger log = LoggerFactory.getLogger(ShellServerIT.class);
 
   private MockShell ts;
 
   private static String rootPath;
 
-  private final SecureRandom random = new SecureRandom();
-
   private static class ShellServerITConfigCallback implements MiniClusterConfigurationCallback {
     @Override
     public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
@@ -261,24 +252,6 @@
     }
   }
 
-  @Test
-  public void setscaniterDeletescaniter() throws Exception {
-    final String table = getUniqueNames(1)[0];
-
-    // setscaniter, deletescaniter
-    ts.exec("createtable " + table);
-    ts.exec("insert a cf cq 1");
-    ts.exec("insert a cf cq 1");
-    ts.exec("insert a cf cq 1");
-    ts.input.set("true\n\n\n\nSTRING");
-    ts.exec("setscaniter -class " + SUMMING_COMBINER_ITERATOR + " -p 10 -n name", true);
-    ts.exec("scan", true, "3", true);
-    ts.exec("deletescaniter -n name", true);
-    ts.exec("scan", true, "1", true);
-    ts.exec("deletetable -f " + table);
-
-  }
-
   @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
   @Test
   public void execfile() throws Exception {
@@ -327,22 +300,6 @@
     ts.exec("deletetable -f " + table);
   }
 
-  /*
-   * This test should be deleted when the debug command is removed
-   */
-  @Deprecated(since = "2.0.0")
-  @Test
-  public void debug() throws Exception {
-    String expectMsg = "The debug command is deprecated";
-    ts.exec("debug", false, expectMsg);
-    ts.exec("debug on", false, expectMsg);
-    ts.exec("debug", false, expectMsg);
-    ts.exec("debug off", false, expectMsg);
-    ts.exec("debug", false, expectMsg);
-    ts.exec("debug debug", false, expectMsg);
-    ts.exec("debug debug debug", false, expectMsg);
-  }
-
   @Test
   public void user() throws Exception {
     final String table = getUniqueNames(1)[0];
@@ -355,14 +312,19 @@
     ts.exec("createuser xyzzy", true);
     ts.exec("users", true, "xyzzy", true);
     String perms = ts.exec("userpermissions -u xyzzy", true);
-    assertTrue(perms.contains("Table permissions (" + MetadataTable.NAME + "): Table.READ"));
+    assertTrue(perms
+        .contains("Table permissions (" + AccumuloTable.METADATA.tableName() + "): Table.READ"));
     ts.exec("grant -u xyzzy -s System.CREATE_TABLE", true);
     perms = ts.exec("userpermissions -u xyzzy", true);
     assertTrue(perms.contains(""));
-    ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.WRITE", true);
-    ts.exec("grant -u " + getPrincipal() + " -t " + MetadataTable.NAME + " Table.GOOFY", false);
+    ts.exec(
+        "grant -u " + getPrincipal() + " -t " + AccumuloTable.METADATA.tableName() + " Table.WRITE",
+        true);
+    ts.exec(
+        "grant -u " + getPrincipal() + " -t " + AccumuloTable.METADATA.tableName() + " Table.GOOFY",
+        false);
     ts.exec("grant -u " + getPrincipal() + " -s foo", false);
-    ts.exec("grant -u xyzzy -t " + MetadataTable.NAME + " foo", false);
+    ts.exec("grant -u xyzzy -t " + AccumuloTable.METADATA.tableName() + " foo", false);
     if (!kerberosEnabled) {
       ts.input.set("secret\nsecret\n");
       ts.exec("user xyzzy", true);
@@ -377,9 +339,9 @@
     ts.exec("revoke -u xyzzy -s System.CREATE_TABLE", true);
     ts.exec("revoke -u xyzzy -s System.GOOFY", false);
     ts.exec("revoke -u xyzzy -s foo", false);
-    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.WRITE", true);
-    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.GOOFY", false);
-    ts.exec("revoke -u xyzzy -t " + MetadataTable.NAME + " foo", false);
+    ts.exec("revoke -u xyzzy -t " + AccumuloTable.METADATA.tableName() + " Table.WRITE", true);
+    ts.exec("revoke -u xyzzy -t " + AccumuloTable.METADATA.tableName() + " Table.GOOFY", false);
+    ts.exec("revoke -u xyzzy -t " + AccumuloTable.METADATA.tableName() + " foo", false);
     ts.exec("deleteuser xyzzy", true, "deleteuser { xyzzy } (yes|no)?", true);
     ts.exec("deleteuser -f xyzzy", true);
     ts.exec("users", true, "xyzzy", false);
@@ -580,7 +542,7 @@
         ts.exec("getauths", true, "bar", true);
         passed = true;
       } catch (AssertionError | Exception e) {
-        sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+        Thread.sleep(500);
       }
     }
     assertTrue(passed, "Could not successfully see updated authoriations");
@@ -647,10 +609,14 @@
 
   @Test
   public void classpath() throws Exception {
-    // classpath
-    ts.exec("classpath", true,
-        "Level: 2, Name: app, class: jdk.internal.loader.ClassLoaders$AppClassLoader: configuration not inspectable",
-        true);
+    final String javaClassPath = System.getProperty("java.class.path");
+
+    // capture classpath from the shell command
+    final String result = ts.exec("classpath", true);
+
+    // for unit tests the classpath should match what the shell returned
+    Arrays.stream(javaClassPath.split(File.pathSeparator))
+        .forEach(classPathUri -> assertTrue(result.contains(classPathUri)));
   }
 
   @Test
@@ -791,14 +757,6 @@
     ts.exec("compact -w");
     assertEquals(1, countFiles(tableId));
 
-    // test compaction strategy
-    ts.exec("insert z 1 2 v900");
-    ts.exec("compact -w -s " + TestCompactionStrategy.class.getName()
-        + " -sc inputPrefix=F,dropPrefix=A");
-    assertEquals(1, countFiles(tableId));
-    ts.exec("scan", true, "v900", true);
-    ts.exec("scan", true, "v901", false);
-
     ts.exec("deletetable -f " + table);
   }
 
@@ -841,7 +799,7 @@
 
     // create two large files
     StringBuilder sb = new StringBuilder("insert b v q ");
-    random.ints(10_000, 0, 26).forEach(i -> sb.append('a' + i));
+    RANDOM.get().ints(10_000, 0, 26).forEach(i -> sb.append('a' + i));
 
     ts.exec(sb.toString());
     ts.exec("flush -w");
@@ -916,18 +874,6 @@
   }
 
   @Test
-  public void testCompactionSelectionAndStrategy() {
-
-    final String table = getUniqueNames(1)[0];
-
-    ts.exec("createtable " + table);
-
-    // expect this to fail
-    ts.exec("compact -t " + table + " -w --sf-ename F.* -s "
-        + TestCompactionStrategy.class.getName() + " -sc inputPrefix=F,dropPrefix=A", false);
-  }
-
-  @Test
   public void testScanSample() throws Exception {
     final String table = getUniqueNames(1)[0];
 
@@ -1056,7 +1002,8 @@
     final String table = getUniqueNames(1)[0];
 
     // constraint
-    ts.exec("constraint -l -t " + MetadataTable.NAME, true, "MetadataConstraints=1", true);
+    ts.exec("constraint -l -t " + AccumuloTable.METADATA.tableName(), true, "MetadataConstraints=1",
+        true);
     ts.exec("createtable " + table + " -evc");
 
     // Make sure the table is fully propagated through zoocache
@@ -1065,7 +1012,7 @@
     ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", true);
     ts.exec("constraint -t " + table + " -d 2", true, "Removed constraint 2 from table " + table);
     // wait for zookeeper updates to propagate
-    sleepUninterruptibly(1, TimeUnit.SECONDS);
+    Thread.sleep(SECONDS.toMillis(1));
     ts.exec("constraint -l -t " + table, true, "VisibilityConstraint=2", false);
     ts.exec("deletetable -f " + table);
   }
@@ -1331,25 +1278,6 @@
     ts.exec("history", true, "deletetable -f " + table, true);
   }
 
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
-  @Test
-  public void importDirectoryOld() throws Exception {
-    final String table = getUniqueNames(1)[0];
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.get(conf);
-    File errorsDir = new File(rootPath, "errors_" + table);
-    assertTrue(errorsDir.mkdir());
-    fs.mkdirs(new Path(errorsDir.toString()));
-
-    String nonce = generateNonce();
-    File importDir = createRFiles(conf, fs, table, nonce);
-    ts.exec("createtable " + table, true);
-    ts.exec("importdirectory " + importDir + " " + errorsDir + " true", true);
-    ts.exec("scan -r 00000000", true, "0-->" + nonce, true);
-    ts.exec("scan -r 00000099", true, "99-->" + nonce, true);
-    ts.exec("deletetable -f " + table);
-  }
-
   @Test
   public void importDirectory() throws Exception {
     final String table = getUniqueNames(1)[0];
@@ -1397,10 +1325,14 @@
     String odd = new File(importDir, "odd.rf").toString();
     AccumuloConfiguration aconf = DefaultConfiguration.getInstance();
     FileSKVWriter evenWriter = FileOperations.getInstance().newWriterBuilder()
-        .forFile(even, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(aconf).build();
+        .forFile(UnreferencedTabletFile.of(fs, new Path(even)), fs, conf,
+            NoCryptoServiceFactory.NONE)
+        .withTableConfiguration(aconf).build();
     evenWriter.startDefaultLocalityGroup();
     FileSKVWriter oddWriter = FileOperations.getInstance().newWriterBuilder()
-        .forFile(odd, fs, conf, NoCryptoServiceFactory.NONE).withTableConfiguration(aconf).build();
+        .forFile(UnreferencedTabletFile.of(fs, new Path(odd)), fs, conf,
+            NoCryptoServiceFactory.NONE)
+        .withTableConfiguration(aconf).build();
     oddWriter.startDefaultLocalityGroup();
     long timestamp = System.currentTimeMillis();
     Text cf = new Text("cf");
@@ -1426,23 +1358,6 @@
   }
 
   @Test
-  public void interpreter() throws Exception {
-    final String table = getUniqueNames(1)[0];
-
-    ts.exec("createtable " + table, true);
-    ts.exec("interpreter -l", true, "HexScan", false);
-    ts.exec("insert \\x02 cf cq value", true);
-    ts.exec("scan -b 02", true, "value", false);
-    ts.exec("interpreter -i org.apache.accumulo.core.util.interpret.HexScanInterpreter", true);
-    // Need to allow time for this to propagate through zoocache/zookeeper
-    sleepUninterruptibly(3, TimeUnit.SECONDS);
-
-    ts.exec("interpreter -l", true, "HexScan", true);
-    ts.exec("scan -b 02", true, "value", true);
-    ts.exec("deletetable -f " + table, true);
-  }
-
-  @Test
   public void listcompactions() throws Exception {
     final String table = getUniqueNames(1)[0];
 
@@ -1491,12 +1406,12 @@
     ts.exec("merge --all", true);
     ts.exec("getsplits", true, "z", false);
     ts.exec("deletetable -f " + table);
-    ts.exec("getsplits -t " + MetadataTable.NAME, true);
+    ts.exec("getsplits -t " + AccumuloTable.METADATA.tableName(), true);
     assertEquals(2, ts.output.get().split("\n").length);
     ts.exec("getsplits -t accumulo.root", true);
     assertEquals(1, ts.output.get().split("\n").length);
-    ts.exec("merge --all -t " + MetadataTable.NAME);
-    ts.exec("getsplits -t " + MetadataTable.NAME, true);
+    ts.exec("merge --all -t " + AccumuloTable.METADATA.tableName());
+    ts.exec("getsplits -t " + AccumuloTable.METADATA.tableName(), true);
     assertEquals(1, ts.output.get().split("\n").length);
   }
 
@@ -1508,7 +1423,7 @@
       if (ts.output.get().split("\n").length == 3) {
         break;
       }
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
+      Thread.sleep(SECONDS.toMillis(1));
 
     }
     assertEquals(2, ts.output.get().split("\n").length);
@@ -1589,7 +1504,7 @@
             log.info("Ignoring scan because of wrong table: {}", currentScan);
           }
         }
-        sleepUninterruptibly(300, TimeUnit.MILLISECONDS);
+        Thread.sleep(300);
       }
       thread.join();
 
@@ -1620,14 +1535,6 @@
   }
 
   @Test
-  public void testPerTableClasspathLegacyJar() throws Exception {
-    final String table = getUniqueNames(1)[0];
-    File fooConstraintJar =
-        initJar("/org/apache/accumulo/test/FooConstraint.jar", "FooContraint", rootPath);
-    verifyPerTableClasspath(table, fooConstraintJar);
-  }
-
-  @Test
   public void testPerTableClasspath_2_1_Jar() throws Exception {
     final String table = getUniqueNames(1)[0];
     File fooConstraintJar =
@@ -1636,18 +1543,18 @@
   }
 
   public void verifyPerTableClasspath(final String table, final File fooConstraintJar)
-      throws IOException {
+      throws IOException, InterruptedException {
 
     File fooFilterJar = initJar("/org/apache/accumulo/test/FooFilter.jar", "FooFilter", rootPath);
 
-    ts.exec("config -s " + VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1=" + fooFilterJar.toURI()
-        + "," + fooConstraintJar.toURI(), true);
+    String context = fooFilterJar.toURI() + "," + fooConstraintJar.toURI();
 
     ts.exec("createtable " + table, true);
-    ts.exec("config -t " + table + " -s " + Property.TABLE_CLASSLOADER_CONTEXT.getKey() + "=cx1",
+    ts.exec(
+        "config -t " + table + " -s " + Property.TABLE_CLASSLOADER_CONTEXT.getKey() + "=" + context,
         true);
 
-    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
+    Thread.sleep(250);
 
     // We can't use the setiter command as Filter implements OptionDescriber which
     // forces us to enter more input that I don't know how to input
@@ -1655,11 +1562,11 @@
     ts.exec("config -t " + table + " -s " + Property.TABLE_ITERATOR_PREFIX.getKey()
         + "scan.foo=10,org.apache.accumulo.test.FooFilter");
 
-    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
+    Thread.sleep(250);
 
     ts.exec("insert foo f q v", true);
 
-    sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
+    Thread.sleep(250);
 
     ts.exec("scan -np", true, "foo", false);
 
@@ -1673,7 +1580,6 @@
     ts.exec("insert ok foo q v", true);
 
     ts.exec("deletetable -f " + table, true);
-    ts.exec("config -d " + VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
 
   }
 
@@ -1820,11 +1726,6 @@
     assertTrue(result.contains("class not found"));
     make10();
     setupFakeContextPath();
-    // Add the context to the table so that setiter works.
-    result = ts.exec("config -s " + VFS_CONTEXT_CLASSPATH_PROPERTY + FAKE_CONTEXT + "="
-        + FAKE_CONTEXT_CLASSPATH);
-    assertEquals("root@miniInstance " + tableName + "> config -s " + VFS_CONTEXT_CLASSPATH_PROPERTY
-        + FAKE_CONTEXT + "=" + FAKE_CONTEXT_CLASSPATH + "\n", result);
 
     result = ts.exec("config -t " + tableName + " -s " + Property.TABLE_CLASSLOADER_CONTEXT.getKey()
         + "=" + FAKE_CONTEXT);
@@ -1855,11 +1756,7 @@
     assertTrue(result.contains("value"));
 
     setupRealContextPath();
-    // Define a new classloader context, but don't set it on the table
-    result = ts.exec("config -s " + VFS_CONTEXT_CLASSPATH_PROPERTY + REAL_CONTEXT + "="
-        + REAL_CONTEXT_CLASSPATH);
-    assertEquals("root@miniInstance " + tableName + "> config -s " + VFS_CONTEXT_CLASSPATH_PROPERTY
-        + REAL_CONTEXT + "=" + REAL_CONTEXT_CLASSPATH + "\n", result);
+
     // Override the table classloader context with the REAL implementation of
     // ValueReversingIterator, which does reverse the value.
     result = ts.exec("scan -pn baz -np -b row1 -e row1 -cc " + REAL_CONTEXT);
@@ -1955,12 +1852,6 @@
 
     ts.exec("createtable " + table, true);
 
-    // validate -t option is used.
-    ts.exec(String.format("importdirectory -t %s %s %s false", table, importDir, errorsDir), true);
-
-    // validate -t option is used.
-    ts.exec(String.format("importdirectory -t %s %s %s false", table, importDir, errorsDir), true);
-
     // validate -t and -i option is used with new bulk import.
     // This will fail as there are no files in the import directory
     ts.exec(String.format("importdirectory -t %s %s false", table, importDir), false);
@@ -1969,12 +1860,8 @@
     // This should pass even if no files in import directory. Empty import dir is ignored.
     ts.exec(String.format("importdirectory -t %s %s false -i", table, importDir), true);
 
-    // validate original cmd format.
-    ts.exec(String.format("table %s", table), true);
-    ts.exec(String.format("importdirectory %s %s false", importDir, errorsDir), true);
-
     // expect fail - invalid command,
-    ts.exec("importdirectory false", false, "Expected 2 or 3 arguments. There was 1.");
+    ts.exec("importdirectory false", false, "Expected 2 arguments. There was 1.");
 
     // expect fail - original cmd without a table.
     ts.exec("notable", true);
@@ -1982,12 +1869,10 @@
         "java.lang.IllegalStateException: Not in a table context.");
   }
 
-  private static final String FAKE_CONTEXT = "FAKE";
-  private static final String FAKE_CONTEXT_CLASSPATH = "file://" + System.getProperty("user.dir")
-      + "/target/" + ShellServerIT.class.getSimpleName() + "-fake-iterators.jar";
-  private static final String REAL_CONTEXT = "REAL";
-  private static final String REAL_CONTEXT_CLASSPATH = "file://" + System.getProperty("user.dir")
-      + "/target/" + ShellServerIT.class.getSimpleName() + "-real-iterators.jar";
+  private static final String FAKE_CONTEXT = "file://" + System.getProperty("user.dir") + "/target/"
+      + ShellServerIT.class.getSimpleName() + "-fake-iterators.jar";
+  private static final String REAL_CONTEXT = "file://" + System.getProperty("user.dir") + "/target/"
+      + ShellServerIT.class.getSimpleName() + "-real-iterators.jar";
   private static final String VALUE_REVERSING_ITERATOR =
       "org.apache.accumulo.test.functional.ValueReversingIterator";
   private static final String SUMMING_COMBINER_ITERATOR =
@@ -2000,7 +1885,7 @@
     Path baseDir = new Path(System.getProperty("user.dir"));
     Path targetDir = new Path(baseDir, "target");
     Path jarPath = new Path(targetDir, "TestJar-Iterators.jar");
-    Path dstPath = new Path(REAL_CONTEXT_CLASSPATH);
+    Path dstPath = new Path(REAL_CONTEXT);
     FileSystem fs = SharedMiniClusterBase.getCluster().getFileSystem();
     fs.copyFromLocalFile(jarPath, dstPath);
   }
@@ -2010,7 +1895,7 @@
     Path baseDir = new Path(System.getProperty("user.dir"));
     Path jarPath = new Path(baseDir + "/target/classes/org/apache/accumulo/test",
         "ShellServerIT-iterators.jar");
-    Path dstPath = new Path(FAKE_CONTEXT_CLASSPATH);
+    Path dstPath = new Path(FAKE_CONTEXT);
     FileSystem fs = SharedMiniClusterBase.getCluster().getFileSystem();
     fs.copyFromLocalFile(jarPath, dstPath);
   }
@@ -2046,8 +1931,8 @@
   private List<String> getFiles(String tableId) {
     ts.output.clear();
 
-    ts.exec(
-        "scan -t " + MetadataTable.NAME + " -np -c file -b " + tableId + " -e " + tableId + "~");
+    ts.exec("scan -t " + AccumuloTable.METADATA.tableName() + " -np -c file -b " + tableId + " -e "
+        + tableId + "~");
 
     log.debug("countFiles(): {}", ts.output.get());
 
@@ -2217,61 +2102,6 @@
   }
 
   @Test
-  public void testFateCommandWithSlowCompaction() throws Exception {
-    final String table = getUniqueNames(1)[0];
-
-    String orgProps = System.getProperty("accumulo.properties");
-
-    System.setProperty("accumulo.properties",
-        "file://" + getCluster().getConfig().getAccumuloPropsFile().getCanonicalPath());
-    // compact
-    ts.exec("createtable " + table);
-
-    // setup SlowIterator to sleep for 10 seconds
-    ts.exec("config -t " + table
-        + " -s table.iterator.majc.slow=1,org.apache.accumulo.test.functional.SlowIterator");
-    ts.exec("config -t " + table + " -s table.iterator.majc.slow.opt.sleepTime=10000");
-
-    // make two files
-    ts.exec("insert a1 b c v_a1");
-    ts.exec("insert a2 b c v_a2");
-    ts.exec("flush -w");
-    ts.exec("insert x1 b c v_x1");
-    ts.exec("insert x2 b c v_x2");
-    ts.exec("flush -w");
-
-    // no transactions running
-    ts.exec("fate -print", true, "0 transactions", true);
-
-    // merge two files into one
-    ts.exec("compact -t " + table);
-    Thread.sleep(1_000);
-    // start 2nd transaction
-    ts.exec("compact -t " + table);
-    Thread.sleep(3_000);
-
-    // 2 compactions should be running so parse the output to get one of the transaction ids
-    log.info("Calling fate print for table = {}", table);
-    String result = ts.exec("fate -print", true, "txid:", true);
-    String[] resultParts = result.split("txid: ");
-    String[] parts = resultParts[1].split(" ");
-    String txid = parts[0];
-    // test filters
-    ts.exec("fate -print -t IN_PROGRESS", true, "2 transactions", true);
-    ts.exec("fate -print " + txid + " -t IN_PROGRESS", true, "1 transactions", true);
-    ts.exec("fate -print " + txid + " -t FAILED", true, "0 transactions", true);
-    ts.exec("fate -print -t NEW", true, "0 transactions", true);
-    ts.exec("fate -print 1234", true, "0 transactions", true);
-    ts.exec("fate -print FATE[aaa] 1 2 3", true, "0 transactions", true);
-
-    ts.exec("deletetable -f " + table);
-
-    if (orgProps != null) {
-      System.setProperty("accumulo.properties", orgProps);
-    }
-  }
-
-  @Test
   public void failOnInvalidClassloaderContestTest() throws Exception {
 
     final String[] names = getUniqueNames(3);
@@ -2300,7 +2130,7 @@
    */
   public String generateNonce() {
     byte[] r = new byte[16];
-    random.nextBytes(r);
+    RANDOM.get().nextBytes(r);
     return new String(Base64.getEncoder().encode(r), UTF_8);
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
index 97b7546..07d45a0 100644
--- a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
@@ -39,7 +39,6 @@
 
 import org.apache.accumulo.compactor.CompactorExecutable;
 import org.apache.accumulo.coordinator.CoordinatorExecutable;
-import org.apache.accumulo.core.file.rfile.CreateEmpty;
 import org.apache.accumulo.core.file.rfile.GenerateSplits;
 import org.apache.accumulo.core.file.rfile.PrintInfo;
 import org.apache.accumulo.core.file.rfile.SplitLarge;
@@ -55,12 +54,10 @@
 import org.apache.accumulo.monitor.MonitorExecutable;
 import org.apache.accumulo.server.conf.CheckCompactionConfig;
 import org.apache.accumulo.server.conf.CheckServerConfig;
-import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
 import org.apache.accumulo.server.conf.util.ZooInfoViewer;
 import org.apache.accumulo.server.conf.util.ZooPropEditor;
 import org.apache.accumulo.server.init.Initialize;
 import org.apache.accumulo.server.util.Admin;
-import org.apache.accumulo.server.util.ConvertConfig;
 import org.apache.accumulo.server.util.DumpZookeeper;
 import org.apache.accumulo.server.util.ECAdmin;
 import org.apache.accumulo.server.util.Info;
@@ -74,6 +71,7 @@
 import org.apache.accumulo.tserver.TServerExecutable;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.logger.LogReader;
+import org.apache.accumulo.tserver.util.CreateEmpty;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
@@ -133,8 +131,6 @@
     expectSet.put("check-server-config", CheckServerConfig.class);
     expectSet.put("compaction-coordinator", CoordinatorExecutable.class);
     expectSet.put("compactor", CompactorExecutable.class);
-    expectSet.put("config-upgrade", ConfigPropertyUpgrader.class);
-    expectSet.put("convert-config", ConvertConfig.class);
     expectSet.put("create-empty", CreateEmpty.class);
     expectSet.put("create-token", CreateToken.class);
     expectSet.put("dump-zoo", DumpZookeeper.class);
@@ -160,10 +156,6 @@
     expectSet.put("zoo-zap", ZooZap.class);
     expectSet.put("zookeeper", ZooKeeperMain.class);
 
-    @SuppressWarnings("deprecation")
-    var masterExecutableClass = org.apache.accumulo.manager.MasterExecutable.class;
-    expectSet.put("master", masterExecutableClass);
-
     Iterator<Entry<String,Class<? extends KeywordExecutable>>> expectIter =
         expectSet.entrySet().iterator();
     TreeMap<String,KeywordExecutable> actualSet = new TreeMap<>(getKeywordExecutables());
@@ -206,8 +198,6 @@
     expectSet.add(Admin.class);
     expectSet.add(CheckCompactionConfig.class);
     expectSet.add(CheckServerConfig.class);
-    expectSet.add(ConfigPropertyUpgrader.class);
-    expectSet.add(ConvertConfig.class);
     expectSet.add(CreateEmpty.class);
     expectSet.add(CreateToken.class);
     expectSet.add(DumpZookeeper.class);
@@ -227,11 +217,6 @@
     expectSet.add(ZooKeeperMain.class);
     expectSet.add(ZooZap.class);
 
-    // not a KeywordExecutable, but this is known to have a main method
-    @SuppressWarnings("deprecation")
-    var masterClass = org.apache.accumulo.master.Master.class;
-    expectSet.add(masterClass);
-
     // check that classes in the expected set contain a main
     // not all have them; these do because they always have, and we don't want to break things
     expectSet.forEach(
diff --git a/test/src/main/java/org/apache/accumulo/test/upgrade/ConfigPropertyUpgraderIT.java b/test/src/main/java/org/apache/accumulo/test/upgrade/ConfigPropertyUpgraderIT.java
deleted file mode 100644
index 51f17f1..0000000
--- a/test/src/main/java/org/apache/accumulo/test/upgrade/ConfigPropertyUpgraderIT.java
+++ /dev/null
@@ -1,153 +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.test.upgrade;
-
-import static org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
-import static org.easymock.EasyMock.createNiceMock;
-import static org.easymock.EasyMock.expect;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.fail;
-
-import java.io.File;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.store.PropStore;
-import org.apache.accumulo.server.conf.store.SystemPropKey;
-import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
-import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
-import org.apache.accumulo.test.conf.store.PropStoreZooKeeperIT;
-import org.apache.accumulo.test.conf.util.LegacyPropData;
-import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Tag;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Tag(ZOOKEEPER_TESTING_SERVER)
-public class ConfigPropertyUpgraderIT {
-
-  private static final Logger log = LoggerFactory.getLogger(PropStoreZooKeeperIT.class);
-  private static ZooKeeperTestingServer testZk = null;
-  private static ZooKeeper zooKeeper;
-  private static ZooReaderWriter zrw;
-
-  private InstanceId instanceId = null;
-
-  @TempDir
-  private static File tempDir;
-
-  @BeforeAll
-  public static void setupZk() {
-
-    // using default zookeeper port - we don't have a full configuration
-    testZk = new ZooKeeperTestingServer(tempDir);
-    zooKeeper = testZk.getZooKeeper();
-    ZooUtil.digestAuth(zooKeeper, ZooKeeperTestingServer.SECRET);
-
-    zrw = testZk.getZooReaderWriter();
-
-  }
-
-  @AfterAll
-  public static void shutdownZK() throws Exception {
-    testZk.close();
-  }
-
-  @BeforeEach
-  public void setupZnodes() throws Exception {
-
-    instanceId = InstanceId.of(UUID.randomUUID());
-
-    testZk.initPaths(ZooUtil.getRoot(instanceId));
-
-    ServerContext context = createNiceMock(ServerContext.class);
-    expect(context.getZooReaderWriter()).andReturn(zrw).anyTimes();
-    expect(context.getZooKeepersSessionTimeOut()).andReturn(zooKeeper.getSessionTimeout())
-        .anyTimes();
-    expect(context.getInstanceID()).andReturn(instanceId).anyTimes();
-
-    List<LegacyPropData.PropNode> nodes = LegacyPropData.getData(instanceId);
-    for (LegacyPropData.PropNode node : nodes) {
-      zrw.putPersistentData(node.getPath(), node.getData(), ZooUtil.NodeExistsPolicy.SKIP);
-    }
-
-    try {
-      zrw.putPersistentData(ZooUtil.getRoot(instanceId) + Constants.ZCONFIG, new byte[0],
-          ZooUtil.NodeExistsPolicy.SKIP);
-    } catch (KeeperException ex) {
-      log.trace("Issue during zk initialization, skipping", ex);
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted during zookeeper path initialization", ex);
-    }
-  }
-
-  @AfterEach
-  public void cleanupZnodes() {
-    try {
-      ZKUtil.deleteRecursive(zooKeeper, Constants.ZROOT);
-    } catch (KeeperException | InterruptedException ex) {
-      throw new IllegalStateException("Failed to clean-up test zooKeeper nodes.", ex);
-    }
-  }
-
-  @Test
-  void doUpgrade() {
-    ConfigPropertyUpgrader upgrader = new ConfigPropertyUpgrader();
-    upgrader.doUpgrade(instanceId, zrw);
-
-    PropStore propStore = ZooPropStore.initialize(instanceId, zrw);
-
-    var sysKey = SystemPropKey.of(instanceId);
-    log.info("PropStore: {}", propStore.get(sysKey));
-
-    var vProps = propStore.get(sysKey);
-    if (vProps == null) {
-      fail("unexpected null returned from prop store get for " + sysKey);
-      return; // keep spotbugs happy
-    }
-
-    Map<String,String> props = vProps.asMap();
-
-    // also validates that rname from deprecated master to manager occured.
-    assertEquals(5, props.size());
-    assertEquals("4", props.get("manager.bulk.retries"));
-    assertEquals("10m", props.get("manager.bulk.timeout"));
-    assertEquals("10", props.get("manager.bulk.rename.threadpool.size"));
-    assertEquals("4", props.get("manager.bulk.threadpool.size"));
-
-    assertEquals("true", props.get("table.bloom.enabled"));
-
-  }
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/upgrade/GCUpgrade9to10TestIT.java b/test/src/main/java/org/apache/accumulo/test/upgrade/GCUpgrade9to10TestIT.java
deleted file mode 100644
index 8977500..0000000
--- a/test/src/main/java/org/apache/accumulo/test/upgrade/GCUpgrade9to10TestIT.java
+++ /dev/null
@@ -1,252 +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.test.upgrade;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.time.Duration;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.metadata.schema.Ample;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.manager.upgrade.Upgrader9to10;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.miniclusterImpl.ProcessNotFoundException;
-import org.apache.accumulo.server.gc.AllVolumesDirectory;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.Text;
-import org.apache.zookeeper.KeeperException;
-import org.junit.jupiter.api.Test;
-
-public class GCUpgrade9to10TestIT extends ConfigurableMacBase {
-  private static final String OUR_SECRET = "itsreallysecret";
-  private static final String OLDDELPREFIX = "~del";
-  private static final Upgrader9to10 upgrader = new Upgrader9to10();
-
-  @Override
-  protected Duration defaultTimeout() {
-    return Duration.ofMinutes(5);
-  }
-
-  @Override
-  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.INSTANCE_SECRET, OUR_SECRET);
-    cfg.setProperty(Property.GC_CYCLE_START, "1000"); // gc will be killed before it is run
-
-    // use raw local file system so walogs sync and flush will work
-    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  private void killMacGc() throws ProcessNotFoundException, InterruptedException, KeeperException {
-    // kill gc started by MAC
-    getCluster().killProcess(ServerType.GARBAGE_COLLECTOR,
-        getCluster().getProcesses().get(ServerType.GARBAGE_COLLECTOR).iterator().next());
-    // delete lock in zookeeper if there, this will allow next GC to start quickly
-    var path = ServiceLock.path(getServerContext().getZooKeeperRoot() + Constants.ZGC_LOCK);
-    ZooReaderWriter zk = getServerContext().getZooReaderWriter();
-    try {
-      ServiceLock.deleteLock(zk, path);
-    } catch (IllegalStateException e) {
-      log.error("Unable to delete ZooLock for mini accumulo-gc", e);
-    }
-
-    assertNull(getCluster().getProcesses().get(ServerType.GARBAGE_COLLECTOR));
-  }
-
-  @Test
-  public void gcUpgradeRootTableDeletesIT() throws Exception {
-    gcUpgradeDeletesTest(Ample.DataLevel.METADATA, 3);
-  }
-
-  @Test
-  public void gcUpgradeMetadataTableDeletesIT() throws Exception {
-    gcUpgradeDeletesTest(Ample.DataLevel.USER, 3);
-  }
-
-  @Test
-  public void gcUpgradeNoDeletesIT() throws Exception {
-    gcUpgradeDeletesTest(Ample.DataLevel.METADATA, 0);
-
-  }
-
-  /**
-   * Ensure that the size of the candidates exceeds the {@link Upgrader9to10}'s CANDIDATE_BATCH_SIZE
-   * and will clean up candidates in multiple batches, without running out of memory.
-   */
-  @Test
-  public void gcUpgradeOutofMemoryTest() throws Exception {
-    killMacGc(); // we do not want anything deleted
-
-    int numberOfEntries = 100_000;
-    String longpathname = StringUtils.repeat("abcde", 100);
-    assertEquals(500, longpathname.length());
-
-    // sanity check to ensure that any batch size assumptions are still valid in this test
-    assertEquals(4_000_000, Upgrader9to10.CANDIDATE_BATCH_SIZE);
-
-    // ensure test quality by making sure we have enough candidates to
-    // exceed the batch size at least ten times
-    long numBatches = numberOfEntries * longpathname.length() / Upgrader9to10.CANDIDATE_BATCH_SIZE;
-    assertTrue(numBatches > 10 && numBatches < 15,
-        "Expected numBatches between 10 and 15, but was " + numBatches);
-
-    Ample.DataLevel level = Ample.DataLevel.USER;
-
-    log.info("Filling metadata table with lots of bogus delete flags");
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-      Map<String,String> expected = addEntries(c, level.metaTable(), numberOfEntries, longpathname);
-      assertEquals(numberOfEntries + numberOfEntries / 10, expected.size());
-
-      Range range = DeletesSection.getRange();
-
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
-      try (Scanner scanner = c.createScanner(level.metaTable(), Authorizations.EMPTY)) {
-        Map<String,String> actualOldStyle = new HashMap<>();
-        scanner.setRange(range);
-        scanner.forEach(entry -> {
-          String strKey = entry.getKey().getRow().toString();
-          String strValue = entry.getValue().toString();
-          actualOldStyle.put(strKey, strValue);
-        });
-        assertEquals(expected.size(), actualOldStyle.size());
-        assertTrue(Collections.disjoint(expected.keySet(), actualOldStyle.keySet()));
-      }
-
-      upgrader.upgradeFileDeletes(getServerContext(), level);
-
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
-      try (Scanner scanner = c.createScanner(level.metaTable(), Authorizations.EMPTY)) {
-        Map<String,String> actualNewStyle = new HashMap<>();
-        scanner.setRange(range);
-        scanner.forEach(entry -> {
-          String strKey = entry.getKey().getRow().toString();
-          String expectedValue = expected.get(strKey);
-          assertNotNull(expectedValue);
-          String strValue = entry.getValue().toString();
-          assertEquals(expectedValue, strValue);
-          actualNewStyle.put(strKey, strValue);
-        });
-        assertEquals(expected.size(), actualNewStyle.size());
-        assertEquals(expected, actualNewStyle);
-      }
-    }
-  }
-
-  private void gcUpgradeDeletesTest(Ample.DataLevel level, int count) throws Exception {
-    killMacGc();// we do not want anything deleted
-
-    log.info("Testing delete upgrades for {}", level.metaTable());
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
-
-      Map<String,String> expected = addEntries(c, level.metaTable(), count, "somefile");
-
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
-      upgrader.upgradeFileDeletes(getServerContext(), level);
-      sleepUninterruptibly(1, TimeUnit.SECONDS);
-      Range range = DeletesSection.getRange();
-
-      try (Scanner scanner = c.createScanner(level.metaTable(), Authorizations.EMPTY)) {
-        Map<String,String> actual = new HashMap<>();
-        scanner.setRange(range);
-        scanner.forEach(entry -> {
-          actual.put(entry.getKey().getRow().toString(), entry.getValue().toString());
-        });
-        assertEquals(expected, actual);
-      }
-
-      // ENSURE IDEMPOTENCE - run upgrade again to ensure nothing is changed because there is
-      // nothing to change
-      upgrader.upgradeFileDeletes(getServerContext(), level);
-      try (Scanner scanner = c.createScanner(level.metaTable(), Authorizations.EMPTY)) {
-        Map<String,String> actual = new HashMap<>();
-        scanner.setRange(range);
-        scanner.forEach(entry -> {
-          actual.put(entry.getKey().getRow().toString(), entry.getValue().toString());
-        });
-        assertEquals(expected, actual);
-      }
-    }
-  }
-
-  private Mutation createOldDelMutation(String path, String cf, String cq, String val) {
-    Text row = new Text(OLDDELPREFIX + path);
-    Mutation delFlag = new Mutation(row);
-    delFlag.put(cf, cq, val);
-    return delFlag;
-  }
-
-  private Map<String,String> addEntries(AccumuloClient client, String table, int count,
-      String filename) throws Exception {
-    client.securityOperations().grantTablePermission(client.whoami(), table, TablePermission.WRITE);
-    Map<String,String> expected = new TreeMap<>();
-    try (BatchWriter bw = client.createBatchWriter(table)) {
-      for (int i = 0; i < count; ++i) {
-        String longpath =
-            String.format("hdfs://localhost:8020/accumulo/tables/5a/t-%08x/%s", i, filename);
-        Mutation delFlag = createOldDelMutation(longpath, "", "", "");
-        bw.addMutation(delFlag);
-        expected.put(DeletesSection.encodeRow(longpath), Upgrader9to10.UPGRADED.toString());
-      }
-
-      // create directory delete entries
-
-      TableId tableId = TableId.of("5a");
-
-      for (int i = 0; i < count; i += 10) {
-        String dirName = String.format("t-%08x", i);
-        String longpath =
-            String.format("hdfs://localhost:8020/accumulo/tables/%s/%s", tableId, dirName);
-        Mutation delFlag = createOldDelMutation(longpath, "", "", "");
-        bw.addMutation(delFlag);
-        expected.put(
-            DeletesSection.encodeRow(new AllVolumesDirectory(tableId, dirName).getMetadataEntry()),
-            Upgrader9to10.UPGRADED.toString());
-      }
-
-      return expected;
-    }
-  }
-
-}
diff --git a/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java b/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java
new file mode 100644
index 0000000..bfc39c9
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java
@@ -0,0 +1,185 @@
+/*
+ * 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.test.util;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+
+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.AbstractTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public class FileMetadataUtil {
+
+  private static final Logger log = LoggerFactory.getLogger(FileMetadataUtil.class);
+
+  public static Map<StoredTabletFile,DataFileValue>
+      printAndVerifyFileMetadata(final ServerContext ctx, TableId tableId) {
+    return printAndVerifyFileMetadata(ctx, tableId, -1);
+  }
+
+  public static Map<StoredTabletFile,DataFileValue>
+      printAndVerifyFileMetadata(final ServerContext ctx, TableId tableId, int expectedFiles) {
+    final Map<StoredTabletFile,DataFileValue> files = new HashMap<>();
+
+    try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+        .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) {
+
+      // Read each file referenced by that table
+      int i = 0;
+      for (TabletMetadata tabletMetadata : tabletsMetadata) {
+        for (Entry<StoredTabletFile,
+            DataFileValue> fileEntry : new TreeMap<>(tabletMetadata.getFilesMap()).entrySet()) {
+          StoredTabletFile file = fileEntry.getKey();
+          DataFileValue dfv = fileEntry.getValue();
+          files.put(file, dfv);
+          log.debug("Extent: " + tabletMetadata.getExtent() + "; File Name: " + file.getFileName()
+              + "; Range: " + file.getRange() + "; Entries: " + dfv.getNumEntries() + ", Size: "
+              + dfv.getSize());
+          i++;
+        }
+      }
+
+      log.debug("");
+      if (expectedFiles >= 0) {
+        assertEquals(expectedFiles, i);
+      }
+    }
+
+    return files;
+  }
+
+  public static int countFiles(final ServerContext ctx, String tableName) {
+    return countFiles(ctx, tableName, null, null);
+  }
+
+  public static int countFiles(final ServerContext ctx, String tableName, Text tabletStartRow,
+      Text tabletEndRow) {
+    final TableId tableId = TableId.of(ctx.tableOperations().tableIdMap().get(tableName));
+    try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+        .overlapping(tabletStartRow, tabletEndRow).fetch(ColumnType.FILES).build()) {
+      return tabletsMetadata.stream().mapToInt(tm -> tm.getFilesMap().size()).sum();
+    }
+  }
+
+  public static int countFencedFiles(final ServerContext ctx, String tableName) {
+    return countFencedFiles(ctx, tableName, null, null);
+  }
+
+  public static int countFencedFiles(final ServerContext ctx, String tableName, Text tabletStartRow,
+      Text tabletEndRow) {
+    final TableId tableId = TableId.of(ctx.tableOperations().tableIdMap().get(tableName));
+    try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId)
+        .overlapping(tabletStartRow, tabletEndRow).fetch(ColumnType.FILES).build()) {
+      return (int) tabletsMetadata.stream().flatMap(tm -> tm.getFilesMap().keySet().stream())
+          .filter(AbstractTabletFile::hasRange).count();
+    }
+  }
+
+  public static void splitFilesIntoRanges(final ServerContext ctx, String tableName,
+      Set<Range> fileRanges) throws Exception {
+    splitFilesIntoRanges(ctx, tableName, null, null, fileRanges);
+  }
+
+  public static void splitFilesIntoRanges(final ServerContext ctx, String tableName,
+      Text tabletStartRow, Text tabletEndRow, Set<Range> fileRanges) throws Exception {
+
+    Preconditions.checkArgument(!fileRanges.isEmpty(), "Ranges must not be empty");
+
+    mutateTabletFiles(ctx, tableName, tabletStartRow, tabletEndRow, (tm, mutator, file, value) -> {
+      // Create a mutation to delete the existing file metadata entry with infinite range
+      mutator.deleteFile(file);
+
+      fileRanges.forEach(range -> {
+        final DataFileValue newValue =
+            new DataFileValue(Integer.max(1, (int) (value.getSize() / fileRanges.size())),
+                Integer.max(1, (int) (value.getNumEntries() / fileRanges.size())));
+        mutator.putFile(StoredTabletFile.of(file.getPath(), range), newValue);
+      });
+    });
+  }
+
+  public static void mutateTabletFiles(final ServerContext ctx, String tableName,
+      Text tabletStartRow, Text tabletEndRow, FileMutator fileMutator) throws Exception {
+
+    final TableId tableId = TableId.of(ctx.tableOperations().tableIdMap().get(tableName));
+
+    // Bring tablet offline so we can modify file metadata
+    ctx.tableOperations().offline(tableName, true);
+
+    try (TabletsMetadata tabletsMetadata =
+        ctx.getAmple().readTablets().forTable(tableId).overlapping(tabletStartRow, tabletEndRow)
+            .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) {
+
+      // Process each tablet in the given start/end row range
+      for (TabletMetadata tabletMetadata : tabletsMetadata) {
+        final KeyExtent ke = tabletMetadata.getExtent();
+
+        TabletMutator mutator = ctx.getAmple().mutateTablet(ke);
+
+        // Read each file and mutate
+        for (Entry<StoredTabletFile,DataFileValue> fileEntry : tabletMetadata.getFilesMap()
+            .entrySet()) {
+          StoredTabletFile file = fileEntry.getKey();
+          DataFileValue value = fileEntry.getValue();
+
+          // do any file mutations
+          fileMutator.mutate(tabletMetadata, mutator, file, value);
+
+          mutator.mutate();
+        }
+      }
+    }
+
+    // Bring back online after metadata updates
+    ctx.tableOperations().online(tableName, true);
+  }
+
+  // Verifies that the MERGED marker was cleared and doesn't exist on any tablet
+  public static void verifyMergedMarkerCleared(final ServerContext ctx, TableId tableId) {
+    try (var tabletsMetadata =
+        ctx.getAmple().readTablets().forTable(tableId).fetch(ColumnType.MERGED).build()) {
+      assertTrue(tabletsMetadata.stream().noneMatch(TabletMetadata::hasMerged));
+    }
+  }
+
+  public interface FileMutator {
+    void mutate(TabletMetadata tm, TabletMutator mutator, StoredTabletFile file,
+        DataFileValue value);
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java b/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java
index 5be70fc..14c291a 100644
--- a/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java
+++ b/test/src/main/java/org/apache/accumulo/test/util/SlowOps.java
@@ -18,6 +18,10 @@
  */
 package org.apache.accumulo.test.util;
 
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -25,7 +29,6 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -42,7 +45,6 @@
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.test.functional.SlowIterator;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -76,14 +78,16 @@
     createData();
   }
 
+  @SuppressWarnings("deprecation")
   public static void setExpectedCompactions(AccumuloClient client, final int numParallelExpected) {
     final int target = numParallelExpected + 1;
     try {
       client.instanceOperations().setProperty(
           Property.TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS.getKey(),
           "[{'name':'any','numThreads':" + target + "}]".replaceAll("'", "\""));
-      UtilWaitThread.sleep(3_000); // give it time to propagate
-    } catch (AccumuloException | AccumuloSecurityException | NumberFormatException ex) {
+      Thread.sleep(3_000); // give it time to propagate
+    } catch (AccumuloException | AccumuloSecurityException | InterruptedException
+        | NumberFormatException ex) {
       throw new IllegalStateException("Could not set parallel compaction limit to " + target, ex);
     }
   }
@@ -116,7 +120,7 @@
     long startTimestamp = System.nanoTime();
     int count = scanCount();
     log.trace("Scan time for {} rows {} ms", NUM_DATA_ROWS,
-        TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimestamp));
+        NANOSECONDS.toMillis(System.nanoTime() - startTimestamp));
     if (count != NUM_DATA_ROWS) {
       throw new IllegalStateException(
           String.format("Number of rows %1$d does not match expected %2$d", count, NUM_DATA_ROWS));
@@ -196,7 +200,7 @@
       log.debug("Compaction wait is complete");
 
       log.trace("Slow compaction of {} rows took {} ms", NUM_DATA_ROWS,
-          TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimestamp));
+          NANOSECONDS.toMillis(System.nanoTime() - startTimestamp));
 
       // validate that number of rows matches expected.
 
@@ -207,7 +211,7 @@
       int count = scanCount();
 
       log.trace("After compaction, scan time for {} rows {} ms", NUM_DATA_ROWS,
-          TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimestamp));
+          NANOSECONDS.toMillis(System.nanoTime() - startTimestamp));
 
       if (count != NUM_DATA_ROWS) {
         throw new IllegalStateException(
@@ -224,7 +228,7 @@
    */
   private boolean blockUntilCompactionRunning() {
     long startWaitNanos = System.nanoTime();
-    long maxWaitNanos = TimeUnit.MILLISECONDS.toNanos(maxWaitMillis);
+    long maxWaitNanos = MILLISECONDS.toNanos(maxWaitMillis);
 
     /*
      * wait for compaction to start on table - The compaction will acquire a fate transaction lock
@@ -257,11 +261,15 @@
         return true;
       }
 
-      UtilWaitThread.sleepUninterruptibly(3, TimeUnit.SECONDS);
+      try {
+        Thread.sleep(SECONDS.toMillis(3));
+      } catch (InterruptedException ex) {
+        throw new IllegalStateException("interrupted during sleep", ex);
+      }
     } while ((System.nanoTime() - startWaitNanos) < maxWaitNanos);
 
     log.debug("Could not find compaction for {} after {} seconds", tableName,
-        TimeUnit.MILLISECONDS.toSeconds(maxWaitMillis));
+        MILLISECONDS.toSeconds(maxWaitMillis));
     return false;
   }
 
diff --git a/test/src/main/resources/log4j2-test.properties b/test/src/main/resources/log4j2-test.properties
index 13ef01c..0c77a38 100644
--- a/test/src/main/resources/log4j2-test.properties
+++ b/test/src/main/resources/log4j2-test.properties
@@ -42,9 +42,6 @@
 logger.05.name = org.apache.accumulo.core.util.shell.Shell
 logger.05.level = fatal
 
-logger.06.name = org.apache.commons.vfs2.impl.DefaultFileSystemManager
-logger.06.level = warn
-
 logger.07.name = org.apache.hadoop.io.compress.CodecPool
 logger.07.level = warn
 
@@ -66,17 +63,14 @@
 logger.13.name = org.apache.zookeeper.ClientCnxn
 logger.13.level = fatal
 
-logger.13b.name = org.apache.zookeeper.ClientCnxnSocket
-logger.13b.level = warn
-
-logger.14.name = org.apache.zookeeper.ZooKeeper
+logger.14.name = org.apache.zookeeper.ClientCnxnSocket
 logger.14.level = warn
 
-logger.15.name = org.apache.accumulo.core.file.rfile.bcfile
-logger.15.level = info
+logger.15.name = org.apache.zookeeper.ZooKeeper
+logger.15.level = warn
 
-logger.16.name = org.apache.accumulo.server.util.ReplicationTableUtil
-logger.16.level = trace
+logger.16.name = org.apache.accumulo.core.file.rfile.bcfile
+logger.16.level = info
 
 logger.17.name = org.apache.accumulo.core.clientImpl.ThriftScanner
 logger.17.level = info
@@ -128,10 +122,6 @@
 logger.33.name = org.apache.hadoop.security.UserGroupInformation
 logger.33.level = info
 
-# For debugging replication tests.
-logger.34.name = org.apache.accumulo.manager.ManagerDrainImpl
-logger.34.level = trace
-
 # This is spammy
 logger.35.name = org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl
 logger.35.level = info
@@ -154,4 +144,3 @@
 
 rootLogger.level = debug
 rootLogger.appenderRef.console.ref = STDOUT
-
diff --git a/test/src/main/resources/org/apache/accumulo/test/FooConstraint.jar b/test/src/main/resources/org/apache/accumulo/test/FooConstraint.jar
deleted file mode 100644
index 14673da..0000000
--- a/test/src/main/resources/org/apache/accumulo/test/FooConstraint.jar
+++ /dev/null
Binary files differ
diff --git a/test/src/main/resources/org/apache/accumulo/test/TestCompactionStrat.jar b/test/src/main/resources/org/apache/accumulo/test/TestCompactionStrat.jar
deleted file mode 100644
index 3daa16e..0000000
--- a/test/src/main/resources/org/apache/accumulo/test/TestCompactionStrat.jar
+++ /dev/null
Binary files differ
diff --git a/test/src/main/resources/v2_import_test/README.md b/test/src/main/resources/v2_import_test/README.md
new file mode 100644
index 0000000..7568d43
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/README.md
@@ -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.
+
+-->
+
+# Import test data created with Accumulo version 2.1.3
+
+This data was created using the Accumulo shell to
+ - create a table
+ - add some splits so there is more than one file (optional)
+ - insert some data
+ - compact the table so the data is written to the tablet's files
+ - offline and export the table to an hdfs directory in the /accumulo namespace
+
+The data can be recreated using the following commands: 
+
+Using hdfs create a directory in the accumulo namespace 
+`hadoop fs -mkdir /accumulo/export_test`
+
+Using the Accumulo shell with any version prior to 3.1
+
+``` 
+> createtable tableA
+> addsplits -t tableA 2 4 6
+> insert -t tableA 1 1
+> insert -t tableA 1 cf cq 1
+> insert -t tableA 2 cf cq 2
+> insert -t tableA 3 cf cq 3
+> insert -t tableA 4 cf cq 4
+> insert -t tableA 5 cf cq 5
+> insert -t tableA 6 cf cq 6
+> insert -t tableA 7 cf cq 7
+
+> compact -w -t tableA
+
+to see the current tablet files: 
+
+> scan -t accumulo.metadata -c file -np
+> offline -t tableA
+
+> exporttable -t tableA /accumulo/export_test
+
+```
+
+The export command will create two files:
+ - /accumulo/export_test/distcp.txt
+ - /accumulo/export_test/exportMetadata.zip
+
+The distcp files lists the tablet files:
+
+```
+>  hadoop fs -cat /accumulo/export_test/distcp.txt
+
+hdfs://localhost:8020/accumulo/tables/1/default_tablet/A000000b.rf
+hdfs://localhost:8020/accumulo/tables/1/t-0000002/A000000a.rf
+hdfs://localhost:8020/accumulo/tables/1/t-0000001/A0000009.rf
+hdfs://localhost:8020/accumulo/tables/1/t-0000003/A0000008.rf
+hdfs://localhost:8020/accumulo/export_test/exportMetadata.zip
+```
+
+The files distcp.txt, exportMetadata.zip and the files listed in distcp where copied from
+hdfs to make this data set.
\ No newline at end of file
diff --git a/test/src/main/resources/v2_import_test/data/A0000008.rf b/test/src/main/resources/v2_import_test/data/A0000008.rf
new file mode 100644
index 0000000..4c333f6
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/data/A0000008.rf
Binary files differ
diff --git a/test/src/main/resources/v2_import_test/data/A0000009.rf b/test/src/main/resources/v2_import_test/data/A0000009.rf
new file mode 100644
index 0000000..fa4df66
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/data/A0000009.rf
Binary files differ
diff --git a/test/src/main/resources/v2_import_test/data/A000000a.rf b/test/src/main/resources/v2_import_test/data/A000000a.rf
new file mode 100644
index 0000000..8a7642c
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/data/A000000a.rf
Binary files differ
diff --git a/test/src/main/resources/v2_import_test/data/A000000b.rf b/test/src/main/resources/v2_import_test/data/A000000b.rf
new file mode 100644
index 0000000..c0503dc
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/data/A000000b.rf
Binary files differ
diff --git a/test/src/main/resources/v2_import_test/data/distcp.txt b/test/src/main/resources/v2_import_test/data/distcp.txt
new file mode 100644
index 0000000..672c11f
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/data/distcp.txt
@@ -0,0 +1,5 @@
+hdfs://localhost:8020/accumulo/tables/1/default_tablet/A000000b.rf
+hdfs://localhost:8020/accumulo/tables/1/t-0000002/A000000a.rf
+hdfs://localhost:8020/accumulo/tables/1/t-0000001/A0000009.rf
+hdfs://localhost:8020/accumulo/tables/1/t-0000003/A0000008.rf
+hdfs://localhost:8020/accumulo/export_test/exportMetadata.zip
diff --git a/test/src/main/resources/v2_import_test/data/exportMetadata.zip b/test/src/main/resources/v2_import_test/data/exportMetadata.zip
new file mode 100644
index 0000000..04494de
--- /dev/null
+++ b/test/src/main/resources/v2_import_test/data/exportMetadata.zip
Binary files differ
diff --git a/test/src/main/spotbugs/exclude-filter.xml b/test/src/main/spotbugs/exclude-filter.xml
index ea7b538..5e0f343 100644
--- a/test/src/main/spotbugs/exclude-filter.xml
+++ b/test/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 -->
diff --git a/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java b/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java
index 90a2646..8b803e7 100644
--- a/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/ChaoticLoadBalancerTest.java
@@ -39,7 +39,7 @@
 import org.apache.accumulo.core.manager.balancer.TServerStatusImpl;
 import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl;
 import org.apache.accumulo.core.manager.balancer.TabletStatisticsImpl;
-import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.core.manager.thrift.TableInfo;
 import org.apache.accumulo.core.spi.balancer.data.TServerStatus;
 import org.apache.accumulo.core.spi.balancer.data.TabletMigration;
 import org.apache.accumulo.core.spi.balancer.data.TabletServerId;
@@ -54,8 +54,8 @@
     List<TabletId> tablets = new ArrayList<>();
 
     TServerStatus getStatus() {
-      org.apache.accumulo.core.master.thrift.TabletServerStatus thriftStatus =
-          new org.apache.accumulo.core.master.thrift.TabletServerStatus();
+      org.apache.accumulo.core.manager.thrift.TabletServerStatus thriftStatus =
+          new org.apache.accumulo.core.manager.thrift.TabletServerStatus();
       thriftStatus.tableMap = new HashMap<>();
       for (TabletId extent : tablets) {
         TableId table = extent.getTable();