Merge branch '2.1'
diff --git a/LICENSE b/LICENSE
index 1c80cd5..22519c1 100644
--- a/LICENSE
+++ b/LICENSE
@@ -291,7 +291,8 @@
 
     Copyright (c) 2007-2014 IOLA and Ole Laursen
 
-    Available under the MIT License (see above and https://github.com/flot/flot/blob/master/LICENSE.txt)
+    Available under the MIT License
+    (see server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/external/flot/LICENSE.txt)
 
     Flot bundles additional works:
 
diff --git a/assemble/bin/accumulo-cluster b/assemble/bin/accumulo-cluster
index d31d659..f525093 100755
--- a/assemble/bin/accumulo-cluster
+++ b/assemble/bin/accumulo-cluster
@@ -49,14 +49,6 @@
 
 function parse_config {
 
-  for file in slaves tservers monitor tracers gc managers masters; do
-    if [[ -f ${conf}/${file} ]]; then
-      echo "ERROR: A '${file}' file was found in ${conf}/"
-      echo "Accumulo now uses cluster host configuration information from 'cluster.yaml' and requires that the '${file}' file not be present to reduce confusion."
-      exit 1
-    fi
-  done
-
   if [[ ! -f ${conf}/cluster.yaml ]]; then
     echo "ERROR: A 'cluster.yaml' file was not found at ${conf}/cluster.yaml"
     echo "Please make sure it exists and is configured with the host information. Run 'accumulo-cluster create-config' to create an example configuration."
@@ -149,16 +141,16 @@
       # using the value of $host
       #
       if [[ $# -gt 3 ]]; then
-        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-a" "$host" "${@:4}"
+        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-o" "general.process.bind.addr=$host" "${@:4}"
       else
-        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-a" "$host"
+        ACCUMULO_SERVICE_INSTANCE="${ACCUMULO_SERVICE_INSTANCE}" "${bin}/accumulo-service" "$service" "$control_cmd" "-o" "general.process.bind.addr=$host"
       fi
     else
       if [[ $# -gt 3 ]]; then
         EXTRA_ARGS="${*:4}"
-        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-a\" \"$host\" $EXTRA_ARGS '"
+        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-o\" \"general.process.bind.addr=$host\" $EXTRA_ARGS '"
       else
-        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-a\" \"$host\"'"
+        $SSH "$host" "bash -c 'ACCUMULO_SERVICE_INSTANCE=${ACCUMULO_SERVICE_INSTANCE} ${bin}/accumulo-service \"$service\" \"$control_cmd\" \"-o\" \"general.process.bind.addr=$host\"'"
       fi
     fi
   done
@@ -204,7 +196,7 @@
   for group in $SSERVER_GROUPS; do
     G="SSERVER_HOSTS_${group}"
     for sserver in ${!G}; do
-      start_service "$sserver" sserver "-g" "$group"
+      start_service "$sserver" sserver "-o" "sserver.group=$group"
     done
   done
 
@@ -215,7 +207,7 @@
   for queue in $COMPACTION_QUEUES; do
     Q="COMPACTOR_HOSTS_${queue}"
     for compactor in ${!Q}; do
-      start_service "$compactor" compactor "-q" "$queue"
+      start_service "$compactor" compactor "-o" "compactor.queue=$queue"
     done
   done
 
@@ -341,7 +333,7 @@
   for group in $SSERVER_GROUPS; do
     G="SSERVER_HOSTS_${group}"
     for sserver in ${!G}; do
-      kill_service "$sserver" sserver "-g" "$group"
+      kill_service "$sserver" sserver "-o" "sserver.group=$group"
     done
   done
 
@@ -356,7 +348,7 @@
   for queue in $COMPACTION_QUEUES; do
     Q="COMPACTOR_HOSTS_${queue}"
     for compactor in ${!Q}; do
-      kill_service "$compactor" compactor "-q" "$queue"
+      kill_service "$compactor" compactor "-o" "compactor.queue=$queue"
     done
   done
 
@@ -397,7 +389,7 @@
     for group in $SSERVER_GROUPS; do
       G="SSERVER_HOSTS_${group}"
       for sserver in ${!G}; do
-        end_service $end_cmd "$sserver" sserver "-g" "$group"
+        end_service $end_cmd "$sserver" sserver "-o" "sserver.group=$group"
       done
     done
 
@@ -408,7 +400,7 @@
     for queue in $COMPACTION_QUEUES; do
       Q="COMPACTOR_HOSTS_${queue}"
       for compactor in ${!Q}; do
-        end_service $end_cmd "$compactor" compactor "-q" "$queue"
+        end_service $end_cmd "$compactor" compactor "-o" "compactor.queue=$queue"
       done
     done
 
@@ -445,13 +437,13 @@
       for group in $SSERVER_GROUPS; do
         G="SSERVER_HOSTS_${group}"
         for sserver in ${!G}; do
-          end_service $end_cmd "$sserver" sserver "-g" "$group"
+          end_service $end_cmd "$sserver" sserver "-o" "sserver.group=$group"
         done
       done
       for queue in $COMPACTION_QUEUES; do
         Q="COMPACTOR_HOSTS_${queue}"
         for compactor in ${!Q}; do
-          end_service $end_cmd "$host" compactor "-q" "$queue"
+          end_service $end_cmd "$host" compactor "-o" "compactor.queue=$queue"
         done
       done
     done
diff --git a/assemble/bin/accumulo-service b/assemble/bin/accumulo-service
index 552c2b2..c501876 100755
--- a/assemble/bin/accumulo-service
+++ b/assemble/bin/accumulo-service
@@ -26,7 +26,6 @@
   gc                     Accumulo garbage collector
   monitor                Accumulo monitor
   manager                Accumulo manager
-  master                 Accumulo master (Deprecated)
   tserver                Accumulo tserver
   compaction-coordinator Accumulo compaction coordinator (experimental)
   compactor              Accumulo compactor (experimental)
@@ -143,11 +142,6 @@
   fi
   service="$1"
 
-  if [[ $service == "master" ]]; then
-    echo "WARN : Use of 'master' service name is deprecated; use 'manager' instead."
-    service="manager"
-  fi
-
   pid_file="${ACCUMULO_PID_DIR}/accumulo-${service}${ACCUMULO_SERVICE_INSTANCE}.pid"
   case "$service" in
     gc | manager | monitor | tserver | compaction-coordinator | compactor | sserver)
diff --git a/assemble/conf/accumulo-env.sh b/assemble/conf/accumulo-env.sh
index bcde690..9e4acc1 100644
--- a/assemble/conf/accumulo-env.sh
+++ b/assemble/conf/accumulo-env.sh
@@ -90,7 +90,7 @@
 # cmd is set by calling script that sources this env file
 #shellcheck disable=SC2154
 case "$cmd" in
-  manager | master) JAVA_OPTS=('-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
+  manager) JAVA_OPTS=('-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
   monitor) JAVA_OPTS=('-Xmx256m' '-Xms256m' "${JAVA_OPTS[@]}") ;;
   gc) JAVA_OPTS=('-Xmx256m' '-Xms256m' "${JAVA_OPTS[@]}") ;;
   tserver) JAVA_OPTS=('-Xmx768m' '-Xms768m' "${JAVA_OPTS[@]}") ;;
@@ -118,7 +118,7 @@
 #JAVA_OPTS=('-javaagent:path/to/opentelemetry-javaagent-all.jar' "${JAVA_OPTS[@]}")
 
 case "$cmd" in
-  monitor | gc | manager | master | tserver | compaction-coordinator | compactor | sserver)
+  monitor | gc | manager | tserver | compaction-coordinator | compactor | sserver)
     JAVA_OPTS=('-Dlog4j.configurationFile=log4j2-service.properties' "${JAVA_OPTS[@]}")
     ;;
   *)
diff --git a/assemble/pom.xml b/assemble/pom.xml
index d24a608..4b7fe16 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>2.1.3-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>accumulo</artifactId>
   <packaging>pom</packaging>
@@ -92,11 +92,6 @@
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
       <groupId>com.lmax</groupId>
       <artifactId>disruptor</artifactId>
       <optional>true</optional>
@@ -188,6 +183,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-access</artifactId>
+      <optional>true</optional>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-compaction-coordinator</artifactId>
       <optional>true</optional>
     </dependency>
@@ -278,11 +278,6 @@
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>org.apache.commons</groupId>
-      <artifactId>commons-vfs2</artifactId>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
       <groupId>org.apache.datasketches</groupId>
       <artifactId>datasketches-java</artifactId>
       <optional>true</optional>
diff --git a/assemble/src/main/resources/LICENSE b/assemble/src/main/resources/LICENSE
index e01f6ae..9bf302b 100644
--- a/assemble/src/main/resources/LICENSE
+++ b/assemble/src/main/resources/LICENSE
@@ -290,7 +290,8 @@
 
     Copyright (c) 2007-2014 IOLA and Ole Laursen
 
-    Available under the MIT License (see above and https://github.com/flot/flot/blob/master/LICENSE.txt)
+    Available under the MIT License
+    (see org/apache/accumulo/monitor/resources/external/flot/LICENSE.txt in the accumulo-monitor jar)
 
     Flot bundles additional works:
 
diff --git a/core/pom.xml b/core/pom.xml
index 9822afe..fcaecd8 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>2.1.3-SNAPSHOT</version>
+    <version>3.1.0-SNAPSHOT</version>
   </parent>
   <artifactId>accumulo-core</artifactId>
   <name>Apache Accumulo Core</name>
@@ -62,10 +62,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
@@ -87,6 +83,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-access</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-start</artifactId>
     </dependency>
     <dependency>
@@ -209,7 +209,6 @@
                   <exclude>src/main/java/org/apache/accumulo/core/bloomfilter/*.java</exclude>
                   <exclude>src/main/java/org/apache/accumulo/core/util/HostAndPort.java</exclude>
                   <exclude>src/test/resources/*.jceks</exclude>
-                  <exclude>src/test/resources/org/apache/accumulo/core/file/rfile/*.rf</exclude>
                 </excludes>
               </licenseSet>
             </licenseSets>
@@ -222,7 +221,6 @@
             <excludes>
               <exclude>src/main/java/org/apache/accumulo/core/bloomfilter/*.java</exclude>
               <exclude>src/test/resources/*.jceks</exclude>
-              <exclude>src/test/resources/org/apache/accumulo/core/file/rfile/*.rf</exclude>
             </excludes>
           </configuration>
         </plugin>
@@ -274,6 +272,8 @@
                 <allow>javax[.]security[.]auth[.]DestroyFailedException</allow>
                 <!-- allow questionable Hadoop exceptions for mapreduce -->
                 <allow>org[.]apache[.]hadoop[.]mapred[.](FileAlreadyExistsException|InvalidJobConfException)</allow>
+                <!-- allow the following types from the visibility API -->
+                <allow>org[.]apache[.]accumulo[.]access[.].*</allow>
               </allows>
             </configuration>
           </execution>
@@ -368,7 +368,7 @@
               <classpathScope>test</classpathScope>
               <arguments>
                 <argument>--generate-markdown</argument>
-                <argument>${project.build.directory}/generated-docs/server-properties.md</argument>
+                <argument>${project.build.directory}/generated-docs/server-properties3.md</argument>
               </arguments>
             </configuration>
           </execution>
@@ -383,7 +383,7 @@
               <classpathScope>test</classpathScope>
               <arguments>
                 <argument>--generate-markdown</argument>
-                <argument>${project.build.directory}/generated-docs/client-properties.md</argument>
+                <argument>${project.build.directory}/generated-docs/client-properties3.md</argument>
               </arguments>
             </configuration>
           </execution>
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index c31e205..411af75 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -35,6 +35,7 @@
   // Zookeeper locations
   public static final String ZROOT = "/accumulo";
   public static final String ZINSTANCES = "/instances";
+  public static final String ZUSERS = "/users";
 
   public static final String ZTABLES = "/tables";
   public static final byte[] ZTABLES_INITIAL_ID = {'0'};
@@ -52,7 +53,6 @@
   public static final String ZMANAGERS = "/managers";
   public static final String ZMANAGER_LOCK = ZMANAGERS + "/lock";
   public static final String ZMANAGER_GOAL_STATE = ZMANAGERS + "/goal_state";
-  public static final String ZMANAGER_REPLICATION_COORDINATOR_ADDR = ZMANAGERS + "/repl_coord_addr";
   public static final String ZMANAGER_TICK = ZMANAGERS + "/tick";
 
   public static final String ZGC = "/gc";
@@ -62,8 +62,6 @@
   public static final String ZMONITOR_LOCK = ZMONITOR + "/lock";
   public static final String ZMONITOR_HTTP_ADDR = ZMONITOR + "/http_addr";
 
-  // used by < 2.1 table and namespace configurations
-  public static final String ZCONF_LEGACY = "/conf";
   public static final String ZCONFIG = "/config";
 
   public static final String ZTSERVERS = "/tservers";
@@ -78,8 +76,6 @@
   public static final String ZDEAD = "/dead";
   public static final String ZDEADTSERVERS = ZDEAD + "/tservers";
 
-  public static final String ZTRACERS = "/tracers";
-
   public static final String ZPROBLEMS = "/problems";
 
   public static final String BULK_ARBITRATOR_TYPE = "bulkTx";
@@ -88,9 +84,6 @@
 
   public static final String ZNEXT_FILE = "/next_file";
 
-  // TODO: Remove when Property.TSERV_WORKQ_THREADS is removed
-  public static final String ZBULK_FAILED_COPYQ = "/bulk_failed_copyq";
-
   public static final String ZHDFS_RESERVATIONS = "/hdfs_reservations";
   public static final String ZRECOVERY = "/recovery";
 
@@ -117,7 +110,6 @@
 
   public static final int MAX_DATA_TO_PRINT = 64;
   public static final String CORE_PACKAGE_NAME = "org.apache.accumulo.core";
-  public static final String MAPFILE_EXTENSION = "map";
   public static final String GENERATED_TABLET_DIRECTORY_PREFIX = "t-";
 
   public static final String EXPORT_METADATA_FILE = "metadata.bin";
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java b/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
index 4928767..dc84a63 100644
--- a/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/classloader/ClassLoaderUtil.java
@@ -45,7 +45,7 @@
         // load the default implementation
         LOG.info("Using default {}, which is subject to change in a future release",
             ContextClassLoaderFactory.class.getName());
-        FACTORY = new DefaultContextClassLoaderFactory(conf);
+        FACTORY = new URLContextClassLoaderFactory();
       } else {
         // load user's selected implementation and provide it with the service environment
         try {
@@ -73,12 +73,15 @@
     FACTORY = null;
   }
 
-  @SuppressWarnings("deprecation")
+  public static ClassLoader getClassLoader() {
+    return getClassLoader(null);
+  }
+
   public static ClassLoader getClassLoader(String context) {
     if (context != null && !context.isEmpty()) {
       return FACTORY.getClassLoader(context);
     } else {
-      return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader.getClassLoader();
+      return ClassLoader.getSystemClassLoader();
     }
   }
 
@@ -114,9 +117,8 @@
   /**
    * Retrieve the classloader context from a table's configuration.
    */
-  @SuppressWarnings("removal")
   public static String tableContext(AccumuloConfiguration conf) {
-    return conf.get(conf.resolve(Property.TABLE_CLASSLOADER_CONTEXT, Property.TABLE_CLASSPATH));
+    return conf.get(Property.TABLE_CLASSLOADER_CONTEXT);
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
deleted file mode 100644
index 152c2b9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/classloader/DefaultContextClassLoaderFactory.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.classloader;
-
-import static java.util.concurrent.TimeUnit.MINUTES;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.core.util.threads.Threads;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The default implementation of ContextClassLoaderFactory. This implementation is subject to change
- * over time. It currently implements the legacy context class loading behavior based on Accumulo's
- * custom class loaders and commons-vfs2. In future, it may simply return the system class loader
- * for all requested contexts. This class is used internally to Accumulo only, and should not be
- * used by users in their configuration.
- */
-public class DefaultContextClassLoaderFactory implements ContextClassLoaderFactory {
-
-  private static final AtomicBoolean isInstantiated = new AtomicBoolean(false);
-  private static final Logger LOG = LoggerFactory.getLogger(DefaultContextClassLoaderFactory.class);
-  private static final String className = DefaultContextClassLoaderFactory.class.getName();
-
-  @SuppressWarnings("removal")
-  private static final Property VFS_CONTEXT_CLASSPATH_PROPERTY =
-      Property.VFS_CONTEXT_CLASSPATH_PROPERTY;
-
-  public DefaultContextClassLoaderFactory(final AccumuloConfiguration accConf) {
-    if (!isInstantiated.compareAndSet(false, true)) {
-      throw new IllegalStateException("Can only instantiate " + className + " once");
-    }
-    Supplier<Map<String,String>> contextConfigSupplier =
-        () -> accConf.getAllPropertiesWithPrefix(VFS_CONTEXT_CLASSPATH_PROPERTY);
-    setContextConfig(contextConfigSupplier);
-    LOG.debug("ContextManager configuration set");
-    startCleanupThread(accConf, contextConfigSupplier);
-  }
-
-  @SuppressWarnings("deprecation")
-  private static void setContextConfig(Supplier<Map<String,String>> contextConfigSupplier) {
-    org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
-        .setContextConfig(contextConfigSupplier);
-  }
-
-  private static void startCleanupThread(final AccumuloConfiguration conf,
-      final Supplier<Map<String,String>> contextConfigSupplier) {
-    ScheduledFuture<?> future = ThreadPools.getClientThreadPools((t, e) -> {
-      LOG.error("context classloader cleanup thread has failed.", e);
-    }).createGeneralScheduledExecutorService(conf)
-        .scheduleWithFixedDelay(Threads.createNamedRunnable(className + "-cleanup", () -> {
-          LOG.trace("{}-cleanup thread, properties: {}", className, conf);
-          Set<String> contextsInUse = contextConfigSupplier.get().keySet().stream()
-              .map(p -> p.substring(VFS_CONTEXT_CLASSPATH_PROPERTY.getKey().length()))
-              .collect(Collectors.toSet());
-          LOG.trace("{}-cleanup thread, contexts in use: {}", className, contextsInUse);
-          removeUnusedContexts(contextsInUse);
-        }), 1, 1, MINUTES);
-    ThreadPools.watchNonCriticalScheduledTask(future);
-    LOG.debug("Context cleanup timer started at 60s intervals");
-  }
-
-  @SuppressWarnings("deprecation")
-  private static void removeUnusedContexts(Set<String> contextsInUse) {
-    org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
-        .removeUnusedContexts(contextsInUse);
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  public ClassLoader getClassLoader(String contextName) {
-    return org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader
-        .getContextClassLoader(contextName);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java b/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java
new file mode 100644
index 0000000..26b06e3
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/classloader/URLContextClassLoaderFactory.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.classloader;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.spi.common.ContextClassLoaderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+
+/**
+ * The default implementation of ContextClassLoaderFactory. This classloader returns a
+ * URLClassLoader based on the given context value which is a CSV list of URLs. For example,
+ * file://path/one/jar1.jar,file://path/two/jar2.jar
+ */
+public class URLContextClassLoaderFactory implements ContextClassLoaderFactory {
+
+  private static final AtomicBoolean isInstantiated = new AtomicBoolean(false);
+  private static final Logger LOG = LoggerFactory.getLogger(URLContextClassLoaderFactory.class);
+  private static final String className = URLContextClassLoaderFactory.class.getName();
+
+  // Cache the class loaders for re-use
+  // WeakReferences are used so that the class loaders can be cleaned up when no longer needed
+  // Classes that are loaded contain a reference to the class loader used to load them
+  // so the class loader will be garbage collected when no more classes are loaded that reference it
+  private final Cache<String,URLClassLoader> classloaders =
+      Caffeine.newBuilder().weakValues().build();
+
+  public URLContextClassLoaderFactory() {
+    if (!isInstantiated.compareAndSet(false, true)) {
+      throw new IllegalStateException("Can only instantiate " + className + " once");
+    }
+  }
+
+  @Override
+  public ClassLoader getClassLoader(String context) {
+    if (context == null) {
+      throw new IllegalArgumentException("Unknown context");
+    }
+
+    return classloaders.get(context, k -> {
+      LOG.debug("Creating URLClassLoader for context, uris: {}", context);
+      return new URLClassLoader(Arrays.stream(context.split(",")).map(url -> {
+        try {
+          return new URL(url);
+        } catch (MalformedURLException e) {
+          throw new RuntimeException(e);
+        }
+      }).toArray(URL[]::new), ClassLoader.getSystemClassLoader());
+    });
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index 25e6ab3..5b82d0e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -51,11 +51,11 @@
  * </pre>
  *
  * <p>
- * If migrating code from Connector to AccumuloClient an important difference to consider is that
- * AccumuloClient is closable and Connector is not. Connector uses static resources and therefore
- * creating them is cheap. AccumuloClient attempts to clean up resources on close, so constantly
- * creating them could perform worse than Connector. Therefore, it would be better to create an
- * AccumuloClient and pass it around.
+ * An important difference with the legacy Connector to consider is that Connector reused global
+ * static resources. AccumuloClient, however, attempts to clean up its resources on close. So,
+ * creating many AccumuloClient objects will perform worse than creating many Connectors did.
+ * Therefore, it is suggested to reuse AccumuloClient instances where possible, rather than create
+ * many of them.
  *
  * <p>
  * AccumuloClient objects are intended to be thread-safe, and can be used by multiple threads.
@@ -292,14 +292,6 @@
   InstanceOperations instanceOperations();
 
   /**
-   * Retrieves a ReplicationOperations object to manage replication configuration.
-   *
-   * @return an object to modify replication configuration
-   */
-  @Deprecated(since = "2.1.0")
-  org.apache.accumulo.core.client.admin.ReplicationOperations replicationOperations();
-
-  /**
    * @return All {@link Properties} used to create client except 'auth.token'
    */
   Properties properties();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
index 68dbb5f..2bf7fcd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriter.java
@@ -23,7 +23,7 @@
 /**
  * Send Mutations to a single Table in Accumulo.
  * <p>
- * When the user uses a Connector to create a BatchWriter, they specify how much memory and how many
+ * When the user uses a client to create a BatchWriter, they specify how much memory and how many
  * threads it should use. As the user adds mutations to the batch writer, it buffers them. Once the
  * buffered mutations have used half of the user specified buffer, the mutations are dumped into the
  * background to be written by a thread pool. If the user specified memory completely fills up, then
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index c570934..d3a8f7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -122,7 +122,6 @@
     } else {
       // make small, positive values that truncate to 0 when converted use the minimum millis
       // instead
-
       this.maxLatency = Math.max(1, timeUnit.toMillis(maxLatency));
     }
     return this;
@@ -157,7 +156,6 @@
     } else {
       // make small, positive values that truncate to 0 when converted use the minimum millis
       // instead
-
       this.timeout = Math.max(1, timeUnit.toMillis(timeout));
     }
     return this;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
deleted file mode 100644
index 89bab19..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.PropertyType;
-import org.apache.commons.configuration2.CompositeConfiguration;
-import org.apache.commons.configuration2.Configuration;
-import org.apache.commons.configuration2.MapConfiguration;
-import org.apache.commons.configuration2.PropertiesConfiguration;
-import org.apache.commons.configuration2.ex.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * Contains a list of property keys recognized by the Accumulo client and convenience methods for
- * setting them.
- *
- * @since 1.6.0
- * @deprecated since 2.0.0, replaced by {@link Accumulo#newClient()}
- */
-@Deprecated(since = "2.0.0")
-public class ClientConfiguration {
-  private static final Logger log = LoggerFactory.getLogger(ClientConfiguration.class);
-
-  public static final String USER_ACCUMULO_DIR_NAME = ".accumulo";
-  public static final String USER_CONF_FILENAME = "config";
-  public static final String GLOBAL_CONF_FILENAME = "client.conf";
-
-  private final CompositeConfiguration compositeConfig;
-
-  public enum ClientProperty {
-    // SSL
-    RPC_SSL_TRUSTSTORE_PATH(Property.RPC_SSL_TRUSTSTORE_PATH),
-    RPC_SSL_TRUSTSTORE_PASSWORD(Property.RPC_SSL_TRUSTSTORE_PASSWORD),
-    RPC_SSL_TRUSTSTORE_TYPE(Property.RPC_SSL_TRUSTSTORE_TYPE),
-    RPC_SSL_KEYSTORE_PATH(Property.RPC_SSL_KEYSTORE_PATH),
-    RPC_SSL_KEYSTORE_PASSWORD(Property.RPC_SSL_KEYSTORE_PASSWORD),
-    RPC_SSL_KEYSTORE_TYPE(Property.RPC_SSL_KEYSTORE_TYPE),
-    RPC_USE_JSSE(Property.RPC_USE_JSSE),
-    GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS),
-    INSTANCE_RPC_SSL_CLIENT_AUTH(Property.INSTANCE_RPC_SSL_CLIENT_AUTH),
-    INSTANCE_RPC_SSL_ENABLED(Property.INSTANCE_RPC_SSL_ENABLED),
-
-    // ZooKeeper
-    INSTANCE_ZK_HOST(Property.INSTANCE_ZK_HOST),
-    INSTANCE_ZK_TIMEOUT(Property.INSTANCE_ZK_TIMEOUT),
-
-    // Instance information
-    INSTANCE_NAME("instance.name", null, PropertyType.STRING,
-        "Name of Accumulo instance to connect to"),
-    INSTANCE_ID("instance.id", null, PropertyType.STRING,
-        "UUID of Accumulo instance to connect to"),
-
-    // Tracing
-    @Deprecated(since = "2.1.0")
-    TRACE_SPAN_RECEIVERS(Property.TRACE_SPAN_RECEIVERS),
-    @Deprecated(since = "2.1.0")
-    TRACE_SPAN_RECEIVER_PREFIX(Property.TRACE_SPAN_RECEIVER_PREFIX),
-    @Deprecated(since = "2.1.0")
-    TRACE_ZK_PATH(Property.TRACE_ZK_PATH),
-
-    // SASL / GSSAPI(Kerberos)
-    /**
-     * @since 1.7.0
-     */
-    INSTANCE_RPC_SASL_ENABLED(Property.INSTANCE_RPC_SASL_ENABLED),
-    /**
-     * @since 1.7.0
-     */
-    RPC_SASL_QOP(Property.RPC_SASL_QOP),
-    /**
-     * @since 1.7.0
-     */
-    KERBEROS_SERVER_PRIMARY("kerberos.server.primary", "accumulo", PropertyType.STRING,
-        "The first component of the Kerberos principal, the 'primary', "
-            + "that Accumulo servers use to login");
-
-    private String key;
-    private String defaultValue;
-    private PropertyType type;
-    private String description;
-
-    private ClientProperty(Property prop) {
-      this(prop.getKey(), prop.getDefaultValue(), prop.getType(), prop.getDescription());
-    }
-
-    private ClientProperty(String key, String defaultValue, PropertyType type, String description) {
-      this.key = key;
-      this.defaultValue = defaultValue;
-      this.type = type;
-      this.description = description;
-    }
-
-    public String getKey() {
-      return key;
-    }
-
-    public String getDefaultValue() {
-      return defaultValue;
-    }
-
-    private PropertyType getType() {
-      return type;
-    }
-
-    public String getDescription() {
-      return description;
-    }
-
-    public static ClientProperty getPropertyByKey(String key) {
-      for (ClientProperty prop : ClientProperty.values()) {
-        if (prop.getKey().equals(key)) {
-          return prop;
-        }
-      }
-      return null;
-    }
-  }
-
-  private ClientConfiguration(List<? extends Configuration> configs) {
-    compositeConfig = new CompositeConfiguration(configs);
-  }
-
-  /**
-   * Attempts to load a configuration file from the system using the default search paths. Uses the
-   * <em>ACCUMULO_CLIENT_CONF_PATH</em> environment variable, split on <em>File.pathSeparator</em>,
-   * for a list of target files.
-   * <p>
-   * If <em>ACCUMULO_CLIENT_CONF_PATH</em> is not set, uses the following in this order:
-   * <ul>
-   * <li>~/.accumulo/config
-   * <li><em>$ACCUMULO_CONF_DIR</em>/client.conf, if <em>$ACCUMULO_CONF_DIR</em> is defined.
-   * <li>/etc/accumulo/client.conf
-   * <li>/etc/accumulo/conf/client.conf
-   * </ul>
-   * <p>
-   * A client configuration will then be read from each location using
-   * <em>PropertiesConfiguration</em> to construct a configuration. That means the latest item will
-   * be the one in the configuration.
-   *
-   * @see PropertiesConfiguration
-   * @see File#pathSeparator
-   */
-  public static ClientConfiguration loadDefault() {
-    return loadFromSearchPath(getDefaultSearchPath());
-  }
-
-  /**
-   * Initializes an empty configuration object to be further configured with other methods on the
-   * class.
-   *
-   * @since 1.9.0
-   */
-  public static ClientConfiguration create() {
-    return new ClientConfiguration(Collections.emptyList());
-  }
-
-  /**
-   * Initializes a configuration object from the contents of a configuration file. Currently
-   * supports Java "properties" files. The returned object can be further configured with subsequent
-   * calls to other methods on this class.
-   *
-   * @param file the path to the configuration file
-   * @since 1.9.0
-   */
-  public static ClientConfiguration fromFile(File file) {
-    var config = new PropertiesConfiguration();
-    try (var reader = new FileReader(file, UTF_8)) {
-      config.read(reader);
-    } catch (ConfigurationException | IOException e) {
-      throw new IllegalArgumentException("Bad configuration file: " + file, e);
-    }
-    return new ClientConfiguration(Collections.singletonList(config));
-  }
-
-  /**
-   * Initializes a configuration object from the contents of a map. The returned object can be
-   * further configured with subsequent calls to other methods on this class.
-   *
-   * @param properties a map containing the configuration properties to use
-   * @since 1.9.0
-   */
-  public static ClientConfiguration fromMap(Map<String,String> properties) {
-    MapConfiguration mapConf = new MapConfiguration(properties);
-    return new ClientConfiguration(Collections.singletonList(mapConf));
-  }
-
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "process runs in same security context as user who provided path")
-  private static ClientConfiguration loadFromSearchPath(List<String> paths) {
-    List<Configuration> configs = new LinkedList<>();
-    for (String path : paths) {
-      File conf = new File(path);
-      if (conf.isFile() && conf.canRead()) {
-        var config = new PropertiesConfiguration();
-        try (var reader = new FileReader(conf, UTF_8)) {
-          config.read(reader);
-        } catch (ConfigurationException | IOException e) {
-          throw new IllegalStateException("Error loading client configuration file " + conf, e);
-        }
-        configs.add(config);
-        log.info("Loaded client configuration file {}", conf);
-      }
-    }
-    // We couldn't find the client configuration anywhere
-    if (configs.isEmpty()) {
-      log.debug(
-          "Found no client.conf in default paths. Using default client configuration values.");
-    }
-    return new ClientConfiguration(configs);
-  }
-
-  public static ClientConfiguration deserialize(String serializedConfig) {
-    var propConfig = new PropertiesConfiguration();
-    try {
-      propConfig.read(new StringReader(serializedConfig));
-    } catch (ConfigurationException | IOException e) {
-      throw new IllegalArgumentException(
-          "Error deserializing client configuration: " + serializedConfig, e);
-    }
-    return new ClientConfiguration(Collections.singletonList(propConfig));
-  }
-
-  /**
-   * Muck the value of {@code clientConfPath} if it points to a directory by appending
-   * {@code client.conf} to the end of the file path. This is a no-op if the value is not a
-   * directory on the filesystem.
-   *
-   * @param clientConfPath The value of ACCUMULO_CLIENT_CONF_PATH.
-   */
-  @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
-      justification = "process runs in same security context as user who provided path")
-  static String getClientConfPath(String clientConfPath) {
-    if (clientConfPath == null) {
-      return null;
-    }
-    File filePath = new File(clientConfPath);
-    // If clientConfPath is a directory, tack on the default client.conf file name.
-    if (filePath.exists() && filePath.isDirectory()) {
-      return new File(filePath, "client.conf").toString();
-    }
-    return clientConfPath;
-  }
-
-  private static List<String> getDefaultSearchPath() {
-    String clientConfSearchPath = getClientConfPath(System.getenv("ACCUMULO_CLIENT_CONF_PATH"));
-    List<String> clientConfPaths;
-    if (clientConfSearchPath != null) {
-      clientConfPaths = Arrays.asList(clientConfSearchPath.split(File.pathSeparator));
-    } else {
-      // if $ACCUMULO_CLIENT_CONF_PATH env isn't set, priority from top to bottom is:
-      // ~/.accumulo/config
-      // $ACCUMULO_CONF_DIR/client.conf
-      // /etc/accumulo/client.conf
-      // /etc/accumulo/conf/client.conf
-      clientConfPaths = new LinkedList<>();
-      clientConfPaths.add(System.getProperty("user.home") + File.separator + USER_ACCUMULO_DIR_NAME
-          + File.separator + USER_CONF_FILENAME);
-      if (System.getenv("ACCUMULO_CONF_DIR") != null) {
-        clientConfPaths
-            .add(System.getenv("ACCUMULO_CONF_DIR") + File.separator + GLOBAL_CONF_FILENAME);
-      }
-      clientConfPaths.add("/etc/accumulo/" + GLOBAL_CONF_FILENAME);
-      clientConfPaths.add("/etc/accumulo/conf/" + GLOBAL_CONF_FILENAME);
-    }
-    return clientConfPaths;
-  }
-
-  public String serialize() {
-    var propConfig = new PropertiesConfiguration();
-    propConfig.copy(compositeConfig);
-    StringWriter writer = new StringWriter();
-    try {
-      propConfig.write(writer);
-    } catch (ConfigurationException | IOException e) {
-      // this should never happen
-      throw new IllegalStateException(e);
-    }
-    return writer.toString();
-  }
-
-  /**
-   * Returns the value for prop, the default value if not present.
-   *
-   */
-  public String get(ClientProperty prop) {
-    if (compositeConfig.containsKey(prop.getKey())) {
-      return compositeConfig.getString(prop.getKey());
-    } else {
-      return prop.getDefaultValue();
-    }
-  }
-
-  private void checkType(ClientProperty property, PropertyType type) {
-    if (!property.getType().equals(type)) {
-      String msg = "Configuration method intended for type " + type + " called with a "
-          + property.getType() + " argument (" + property.getKey() + ")";
-      throw new IllegalArgumentException(msg);
-    }
-  }
-
-  /**
-   * Gets all properties under the given prefix in this configuration.
-   *
-   * @param property prefix property, must be of type PropertyType.PREFIX
-   * @return a map of property keys to values
-   * @throws IllegalArgumentException if property is not a prefix
-   */
-  public Map<String,String> getAllPropertiesWithPrefix(ClientProperty property) {
-    checkType(property, PropertyType.PREFIX);
-
-    Map<String,String> propMap = new HashMap<>();
-    String prefix = property.getKey();
-    if (prefix.endsWith(".")) {
-      prefix = prefix.substring(0, prefix.length() - 1);
-    }
-    Iterator<?> iter = compositeConfig.getKeys(prefix);
-    while (iter.hasNext()) {
-      String p = (String) iter.next();
-      propMap.put(p, compositeConfig.getString(p));
-    }
-    return propMap;
-  }
-
-  /**
-   * Sets the value of property to value
-   *
-   */
-  public void setProperty(ClientProperty prop, String value) {
-    with(prop, value);
-  }
-
-  /**
-   * Same as {@link #setProperty(ClientProperty, String)} but returns the ClientConfiguration for
-   * chaining purposes
-   */
-  public ClientConfiguration with(ClientProperty prop, String value) {
-    return with(prop.getKey(), value);
-  }
-
-  /**
-   * Sets the value of property to value
-   *
-   * @since 1.9.0
-   */
-  public void setProperty(String prop, String value) {
-    with(prop, value);
-  }
-
-  /**
-   * Same as {@link #setProperty(String, String)} but returns the ClientConfiguration for chaining
-   * purposes
-   *
-   * @since 1.9.0
-   */
-  public ClientConfiguration with(String prop, String value) {
-    compositeConfig.setProperty(prop, value);
-    return this;
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_NAME
-   *
-   */
-  public ClientConfiguration withInstance(String instanceName) {
-    checkArgument(instanceName != null, "instanceName is null");
-    return with(ClientProperty.INSTANCE_NAME, instanceName);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ID
-   *
-   */
-  public ClientConfiguration withInstance(UUID instanceId) {
-    checkArgument(instanceId != null, "instanceId is null");
-    return with(ClientProperty.INSTANCE_ID, instanceId.toString());
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_HOST
-   *
-   */
-  public ClientConfiguration withZkHosts(String zooKeepers) {
-    checkArgument(zooKeepers != null, "zooKeepers is null");
-    return with(ClientProperty.INSTANCE_ZK_HOST, zooKeepers);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_ZK_TIMEOUT
-   *
-   */
-  public ClientConfiguration withZkTimeout(int timeout) {
-    return with(ClientProperty.INSTANCE_ZK_TIMEOUT, String.valueOf(timeout));
-  }
-
-  /**
-   * Same as {@link #withSsl(boolean, boolean)} with useJsseConfig set to false
-   *
-   */
-  public ClientConfiguration withSsl(boolean sslEnabled) {
-    return withSsl(sslEnabled, false);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_ENABLED and
-   * ClientProperty.RPC_USE_JSSE
-   *
-   */
-  public ClientConfiguration withSsl(boolean sslEnabled, boolean useJsseConfig) {
-    return with(ClientProperty.INSTANCE_RPC_SSL_ENABLED, String.valueOf(sslEnabled))
-        .with(ClientProperty.RPC_USE_JSSE, String.valueOf(useJsseConfig));
-  }
-
-  /**
-   * Same as {@link #withTruststore(String, String, String)} with password null and type null
-   *
-   */
-  public ClientConfiguration withTruststore(String path) {
-    return withTruststore(path, null, null);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.RPC_SSL_TRUSTORE_PATH,
-   * ClientProperty.RPC_SSL_TRUSTORE_PASSWORD, and ClientProperty.RPC_SSL_TRUSTORE_TYPE
-   *
-   */
-  public ClientConfiguration withTruststore(String path, String password, String type) {
-    checkArgument(path != null, "path is null");
-    setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PATH, path);
-    if (password != null) {
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD, password);
-    }
-    if (type != null) {
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, type);
-    }
-    return this;
-  }
-
-  /**
-   * Same as {@link #withKeystore(String, String, String)} with password null and type null
-   *
-   */
-  public ClientConfiguration withKeystore(String path) {
-    return withKeystore(path, null, null);
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-   * ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, and
-   * ClientProperty.RPC_SSL_KEYSTORE_TYPE
-   *
-   */
-  public ClientConfiguration withKeystore(String path, String password, String type) {
-    checkArgument(path != null, "path is null");
-    setProperty(ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH, "true");
-    setProperty(ClientProperty.RPC_SSL_KEYSTORE_PATH, path);
-    if (password != null) {
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, password);
-    }
-    if (type != null) {
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_TYPE, type);
-    }
-    return this;
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED.
-   *
-   * @since 1.7.0
-   */
-  public ClientConfiguration withSasl(boolean saslEnabled) {
-    return with(ClientProperty.INSTANCE_RPC_SASL_ENABLED, String.valueOf(saslEnabled));
-  }
-
-  /**
-   * Show whether SASL has been set on this configuration.
-   *
-   * @since 1.9.0
-   */
-  public boolean hasSasl() {
-    return compositeConfig.getBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getKey(),
-        Boolean.parseBoolean(ClientProperty.INSTANCE_RPC_SASL_ENABLED.getDefaultValue()));
-  }
-
-  /**
-   * Same as {@link #with(ClientProperty, String)} for ClientProperty.INSTANCE_RPC_SASL_ENABLED and
-   * ClientProperty.GENERAL_KERBEROS_PRINCIPAL.
-   *
-   * @param saslEnabled Should SASL(kerberos) be enabled
-   * @param kerberosServerPrimary The 'primary' component of the Kerberos principal Accumulo servers
-   *        use to login (e.g. 'accumulo' in 'accumulo/_HOST@REALM')
-   * @since 1.7.0
-   */
-  public ClientConfiguration withSasl(boolean saslEnabled, String kerberosServerPrimary) {
-    return withSasl(saslEnabled).with(ClientProperty.KERBEROS_SERVER_PRIMARY,
-        kerberosServerPrimary);
-  }
-
-  public boolean containsKey(String key) {
-    return compositeConfig.containsKey(key);
-  }
-
-  public Iterator<String> getKeys() {
-    return compositeConfig.getKeys();
-  }
-
-  public String getString(String key) {
-    return compositeConfig.getString(key);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index 362c4e8..7e300e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -22,6 +22,7 @@
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
@@ -52,7 +53,6 @@
 import org.apache.accumulo.core.iteratorsImpl.IteratorBuilder;
 import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -134,12 +134,6 @@
       return samplerConfig;
     }
 
-    @Deprecated(since = "2.1.0")
-    @Override
-    public ServiceEnvironment getServiceEnv() {
-      return new ClientServiceEnvironmentImpl(context.get());
-    }
-
     @Override
     public PluginEnvironment getPluginEnv() {
       return new ClientServiceEnvironmentImpl(context.get());
@@ -303,7 +297,7 @@
 
       skvi = IteratorConfigUtil.loadIterators(smi, ib);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
 
     final Set<ByteSequence> colfs = new TreeSet<>();
@@ -314,7 +308,7 @@
     try {
       skvi.seek(range, colfs, true);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
 
     return new IteratorAdapter(skvi);
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
deleted file mode 100644
index 990824d..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ConnectorImpl;
-import org.apache.accumulo.core.security.Authorizations;
-
-/**
- * Connector connects to an Accumulo instance and allows the user to request readers and writers for
- * the instance as well as various objects that permit administrative operations.
- *
- * The Connector enforces security on the client side by forcing all API calls to be accompanied by
- * user credentials.
- *
- * @deprecated since 2.0.0. Use {@link AccumuloClient} for writing new code. Connector is available
- *             for existing code. Use {@link #from(AccumuloClient)} as a bridge between the two.
- */
-@Deprecated(since = "2.0.0")
-public abstract class Connector {
-
-  /**
-   * Factory method to create a BatchScanner connected to Accumulo.
-   *
-   * @param tableName the name of the table to query
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   * @param numQueryThreads the number of concurrent threads to spawn for querying
-   *
-   * @return BatchScanner object for configuring and querying
-   * @throws TableNotFoundException when the specified table doesn't exist
-   */
-  public abstract BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
-      int numQueryThreads) throws TableNotFoundException;
-
-  /**
-   * Factory method to create a BatchDeleter connected to Accumulo.
-   *
-   * @param tableName the name of the table to query and delete from
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   * @param numQueryThreads the number of concurrent threads to spawn for querying
-   * @param maxMemory size in bytes of the maximum memory to batch before writing
-   * @param maxLatency size in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
-   *        hold a batch before writing
-   * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
-   *        servers
-   *
-   * @return BatchDeleter object for configuring and deleting
-   * @throws TableNotFoundException when the specified table doesn't exist
-   * @deprecated since 1.5.0; Use
-   *             {@link #createBatchDeleter(String, Authorizations, int, BatchWriterConfig)}
-   *             instead.
-   */
-  @Deprecated(since = "1.5.0")
-  public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
-      throws TableNotFoundException;
-
-  /**
-   * Factory method to create BatchDeleter
-   *
-   * @param tableName the name of the table to query and delete from
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   * @param numQueryThreads the number of concurrent threads to spawn for querying
-   * @param config configuration used to create batch writer. This config takes precedence. Any
-   *        unset values will be merged with config set when the Connector was created. If no config
-   *        was set during Connector creation, BatchWriterConfig defaults will be used.
-   * @return BatchDeleter object for configuring and deleting
-   * @since 1.5.0
-   */
-  public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException;
-
-  /**
-   * Factory method to create a BatchWriter connected to Accumulo.
-   *
-   * @param tableName the name of the table to insert data into
-   * @param maxMemory size in bytes of the maximum memory to batch before writing
-   * @param maxLatency time in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
-   *        hold a batch before writing
-   * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
-   *        servers
-   *
-   * @return BatchWriter object for configuring and writing data to
-   * @throws TableNotFoundException when the specified table doesn't exist
-   * @deprecated since 1.5.0; Use {@link #createBatchWriter(String, BatchWriterConfig)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  public abstract BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
-      int maxWriteThreads) throws TableNotFoundException;
-
-  /**
-   * Factory method to create a BatchWriter connected to Accumulo.
-   *
-   * @param tableName the name of the table to insert data into
-   * @param config configuration used to create batch writer. This config will take precedence. Any
-   *        unset values will merged with config set when the Connector was created. If no config
-   *        was set during Connector creation, BatchWriterConfig defaults will be used.
-   * @return BatchWriter object for configuring and writing data to
-   * @since 1.5.0
-   */
-  public abstract BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
-      throws TableNotFoundException;
-
-  /**
-   * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch
-   * writers can queue data for multiple tables, which is good for ingesting data into multiple
-   * tables from the same source
-   *
-   * @param maxMemory size in bytes of the maximum memory to batch before writing
-   * @param maxLatency size in milliseconds; set to 0 or Long.MAX_VALUE to allow the maximum time to
-   *        hold a batch before writing
-   * @param maxWriteThreads the maximum number of threads to use for writing data to the tablet
-   *        servers
-   *
-   * @return MultiTableBatchWriter object for configuring and writing data to
-   * @deprecated since 1.5.0; Use {@link #createMultiTableBatchWriter(BatchWriterConfig)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  public abstract MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
-      int maxWriteThreads);
-
-  /**
-   * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch
-   * writers can queue data for multiple tables. Also data for multiple tables can be sent to a
-   * server in a single batch. Its an efficient way to ingest data into multiple tables from a
-   * single process.
-   *
-   * @param config configuration used to create multi-table batch writer. This config will take
-   *        precedence. Any unset values will merged with config set when the Connector was created.
-   *        If no config was set during Connector creation, BatchWriterConfig defaults will be used.
-   * @return MultiTableBatchWriter object for configuring and writing data to
-   * @since 1.5.0
-   */
-  public abstract MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config);
-
-  /**
-   * Factory method to create a Scanner connected to Accumulo.
-   *
-   * @param tableName the name of the table to query data from
-   * @param authorizations A set of authorization labels that will be checked against the column
-   *        visibility of each key in order to filter data. The authorizations passed in must be a
-   *        subset of the accumulo user's set of authorizations. If the accumulo user has
-   *        authorizations (A1, A2) and authorizations (A2, A3) are passed, then an exception will
-   *        be thrown.
-   *
-   * @return Scanner object for configuring and querying data with
-   * @throws TableNotFoundException when the specified table doesn't exist
-   */
-  public abstract Scanner createScanner(String tableName, Authorizations authorizations)
-      throws TableNotFoundException;
-
-  /**
-   * Factory method to create a ConditionalWriter connected to Accumulo.
-   *
-   * @param tableName the name of the table to query data from
-   * @param config configuration used to create conditional writer
-   *
-   * @return ConditionalWriter object for writing ConditionalMutations
-   * @throws TableNotFoundException when the specified table doesn't exist
-   * @since 1.6.0
-   */
-  public abstract ConditionalWriter createConditionalWriter(String tableName,
-      ConditionalWriterConfig config) throws TableNotFoundException;
-
-  /**
-   * Accessor method for internal instance object.
-   *
-   * @return the internal instance object
-   */
-  public abstract Instance getInstance();
-
-  /**
-   * Get the current user for this connector
-   *
-   * @return the user name
-   */
-  public abstract String whoami();
-
-  /**
-   * Retrieves a TableOperations object to perform table functions, such as create and delete.
-   *
-   * @return an object to manipulate tables
-   */
-  public abstract TableOperations tableOperations();
-
-  /**
-   * Retrieves a NamespaceOperations object to perform namespace functions, such as create and
-   * delete.
-   *
-   * @return an object to manipulate namespaces
-   */
-  public abstract NamespaceOperations namespaceOperations();
-
-  /**
-   * Retrieves a SecurityOperations object to perform user security operations, such as creating
-   * users.
-   *
-   * @return an object to modify users and permissions
-   */
-  public abstract SecurityOperations securityOperations();
-
-  /**
-   * Retrieves an InstanceOperations object to modify instance configuration.
-   *
-   * @return an object to modify instance configuration
-   */
-  public abstract InstanceOperations instanceOperations();
-
-  /**
-   * Retrieves a ReplicationOperations object to manage replication configuration.
-   *
-   * @return an object to modify replication configuration
-   * @since 1.7.0
-   */
-  public abstract ReplicationOperations replicationOperations();
-
-  /**
-   * Creates a Connector from an AccumuloClient. This Connector will no longer work after the
-   * AccumuloClient is closed. Also anything derived from the Connector (like a Scanner for example)
-   * is unlikely to work after the AccumuloClient is closed.
-   *
-   * @since 2.0
-   */
-  public static Connector from(AccumuloClient client)
-      throws AccumuloSecurityException, AccumuloException {
-    return new ConnectorImpl((ClientContext) client);
-  }
-
-  /**
-   * Creates a new Accumulo Client from a Connector. The returned client should be closed and
-   * closing it will not affect the Connector from which it was derived. This method is useful for
-   * cases where code written using Connector must call code written using AccumuloClient. Below is
-   * an example.
-   *
-   * <pre>
-   * <code>
-   *   Connector conn = getMyConnector();
-   *   try(AccumuloClient client = Connector.newClient(conn) {
-   *      doSomething(client);
-   *   }
-   * </code>
-   * </pre>
-   *
-   * @since 2.1.0
-   */
-  public static AccumuloClient newClient(Connector conn) {
-    return Accumulo.newClient().from(((ConnectorImpl) conn).getAccumuloClient().getProperties())
-        .build();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
deleted file mode 100644
index 947f8b1..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-
-/**
- * This class represents the information a client needs to know to connect to an instance of
- * accumulo.
- *
- * @deprecated since 2.0.0, use {@link Accumulo#newClient()} and {@link java.util.Properties}
- *             instead
- */
-@Deprecated(since = "2.0.0")
-public interface Instance {
-  /**
-   * Returns the location of the tablet server that is serving the root tablet.
-   *
-   * @return location in "hostname:port" form
-   */
-  String getRootTabletLocation();
-
-  /**
-   * Returns the location(s) of the accumulo manager and any redundant servers.
-   *
-   * @return a list of locations in "hostname:port" form
-   */
-  List<String> getMasterLocations();
-
-  /**
-   * Returns a unique string that identifies this instance of accumulo.
-   *
-   * @return a UUID
-   */
-  String getInstanceID();
-
-  /**
-   * Returns the instance name given at system initialization time.
-   *
-   * @return current instance name
-   */
-  String getInstanceName();
-
-  /**
-   * Returns a comma-separated list of zookeeper servers the instance is using.
-   *
-   * @return the zookeeper servers this instance is using in "hostname:port" form
-   */
-  String getZooKeepers();
-
-  /**
-   * Returns the zookeeper connection timeout.
-   *
-   * @return the configured timeout to connect to zookeeper
-   */
-  int getZooKeepersSessionTimeOut();
-
-  /**
-   * Returns a connection to accumulo.
-   *
-   * @param user a valid accumulo user
-   * @param pass A UTF-8 encoded password. The password may be cleared after making this call.
-   * @return the accumulo Connector
-   * @throws AccumuloException when a generic exception occurs
-   * @throws AccumuloSecurityException when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
-   *             {@link PasswordToken}
-   */
-  @Deprecated(since = "1.5.0")
-  default Connector getConnector(String user, byte[] pass)
-      throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken(pass));
-  }
-
-  /**
-   * Returns a connection to accumulo.
-   *
-   * @param user a valid accumulo user
-   * @param pass A UTF-8 encoded password. The password may be cleared after making this call.
-   * @return the accumulo Connector
-   * @throws AccumuloException when a generic exception occurs
-   * @throws AccumuloSecurityException when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
-   *             {@link PasswordToken}
-   */
-  @Deprecated(since = "1.5.0")
-  default Connector getConnector(String user, ByteBuffer pass)
-      throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken(pass));
-  }
-
-  /**
-   * Returns a connection to this instance of accumulo.
-   *
-   * @param user a valid accumulo user
-   * @param pass If a mutable CharSequence is passed in, it may be cleared after this call.
-   * @return the accumulo Connector
-   * @throws AccumuloException when a generic exception occurs
-   * @throws AccumuloSecurityException when a user's credentials are invalid
-   * @deprecated since 1.5, use {@link #getConnector(String, AuthenticationToken)} with
-   *             {@link PasswordToken}
-   */
-  @Deprecated(since = "1.5.0")
-  default Connector getConnector(String user, CharSequence pass)
-      throws AccumuloException, AccumuloSecurityException {
-    return getConnector(user, new PasswordToken(pass));
-  }
-
-  /**
-   * Returns a connection to this instance of accumulo.
-   *
-   * @param principal a valid accumulo user
-   * @param token Use the token type configured for the Accumulo instance you are connecting to. An
-   *        Accumulo instance with default configurations will use {@link PasswordToken}
-   * @since 1.5.0
-   */
-  Connector getConnector(String principal, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
index 518737b..c3ead89 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java
@@ -18,8 +18,8 @@
  */
 package org.apache.accumulo.core.client;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.ArrayList;
 import java.util.Iterator;
diff --git a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
index 30e6fbd..1603268 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/MutationsRejectedException.java
@@ -45,34 +45,6 @@
   private final int unknownErrors;
 
   /**
-   *
-   * @param cvsList list of constraint violations
-   * @param hashMap authorization failures
-   * @param serverSideErrors server side errors
-   * @param unknownErrors number of unknown errors
-   *
-   * @since 1.7.0
-   * @deprecated since 2.0.0, replaced by
-   *             {@link #MutationsRejectedException(AccumuloClient, List, Map, Collection, int, Throwable)}
-   */
-  @Deprecated(since = "2.0.0")
-  public MutationsRejectedException(Instance instance, List<ConstraintViolationSummary> cvsList,
-      Map<TabletId,Set<SecurityErrorCode>> hashMap, Collection<String> serverSideErrors,
-      int unknownErrors, Throwable cause) {
-    super(
-        "constraint violation codes : "
-            + cvsList.stream().map(ConstraintViolationSummary::getViolationCode)
-                .collect(Collectors.toSet())
-            + "  security codes: " + hashMap.toString() + "  # server errors "
-            + serverSideErrors.size() + " # exceptions " + unknownErrors,
-        cause);
-    this.cvsl.addAll(cvsList);
-    this.af.putAll(hashMap);
-    this.es.addAll(serverSideErrors);
-    this.unknownErrors = unknownErrors;
-  }
-
-  /**
    * Creates Mutations rejected exception
    *
    * @param client AccumuloClient
diff --git a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
index 6653c80..4372a15 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/PluginEnvironment.java
@@ -163,7 +163,7 @@
    * @param className Fully qualified name of the class.
    * @param base The expected super type of the class.
    */
-  <T> T instantiate(String className, Class<T> base) throws Exception;
+  <T> T instantiate(String className, Class<T> base) throws ReflectiveOperationException;
 
   /**
    * Instantiate a class using Accumulo's per table classloader. The class must have a no argument
@@ -172,5 +172,6 @@
    * @param className Fully qualified name of the class.
    * @param base The expected super type of the class.
    */
-  <T> T instantiate(TableId tableId, String className, Class<T> base) throws Exception;
+  <T> T instantiate(TableId tableId, String className, Class<T> base)
+      throws ReflectiveOperationException;
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
index 2dbeb14..bd0e266 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableOfflineException.java
@@ -25,14 +25,6 @@
   private static final long serialVersionUID = 1L;
 
   /**
-   * @deprecated since 2.0.0, replaced by {@link #TableOfflineException(String)}
-   */
-  @Deprecated(since = "2.0.0")
-  public TableOfflineException(Instance instance, String tableId) {
-    super("Table with ID (" + tableId + ") is offline");
-  }
-
-  /**
    * @since 2.0.0
    */
   public TableOfflineException(String msg) {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
deleted file mode 100644
index aa96404..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.util.concurrent.TimeUnit.SECONDS;
-
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.ClientConfConverter;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
-import org.apache.accumulo.core.clientImpl.InstanceOperationsImpl;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.fate.zookeeper.ZooCache;
-import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
-import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
-import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.core.singletons.SingletonReservation;
-import org.apache.accumulo.core.util.OpTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation of instance that looks in zookeeper to find information needed to connect to an
- * instance of accumulo.
- *
- * <p>
- * The advantage of using zookeeper to obtain information about accumulo is that zookeeper is highly
- * available, very responsive, and supports caching.
- *
- * <p>
- * Because it is possible for multiple instances of accumulo to share a single set of zookeeper
- * servers, all constructors require an accumulo instance name.
- *
- * If you do not know the instance names then run accumulo
- * org.apache.accumulo.server.util.ListInstances on an accumulo server.
- *
- * @deprecated since 2.0.0, Use {@link Accumulo#newClient()} instead
- */
-@Deprecated(since = "2.0.0")
-public class ZooKeeperInstance implements Instance {
-
-  private static final Logger log = LoggerFactory.getLogger(ZooKeeperInstance.class);
-
-  private String instanceId = null;
-  private String instanceName = null;
-
-  private final ZooCache zooCache;
-
-  private final String zooKeepers;
-
-  private final int zooKeepersSessionTimeOut;
-
-  private ClientConfiguration clientConf;
-
-  /**
-   *
-   * @param instanceName The name of specific accumulo instance. This is set at initialization time.
-   * @param zooKeepers A comma separated list of zoo keeper server locations. Each location can
-   *        contain an optional port, of the format host:port.
-   */
-  public ZooKeeperInstance(String instanceName, String zooKeepers) {
-    this(ClientConfiguration.loadDefault().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  ZooKeeperInstance(ClientConfiguration config, ZooCacheFactory zcf) {
-    checkArgument(config != null, "config is null");
-    // Enable singletons before before getting a zoocache
-    SingletonManager.setMode(Mode.CONNECTOR);
-    this.clientConf = config;
-    this.instanceId = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ID);
-    this.instanceName = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_NAME);
-    if ((instanceId == null) == (instanceName == null)) {
-      throw new IllegalArgumentException(
-          "Expected exactly one of instanceName and instanceId to be set; "
-              + (instanceName == null ? "neither" : "both") + " were set");
-    }
-    this.zooKeepers = clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST);
-    this.zooKeepersSessionTimeOut = (int) ConfigurationTypeHelper
-        .getTimeInMillis(clientConf.get(ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT));
-    zooCache = zcf.getZooCache(zooKeepers, zooKeepersSessionTimeOut);
-    if (instanceName != null) {
-      // Validates that the provided instanceName actually exists
-      getInstanceID();
-    }
-  }
-
-  /**
-   * @param config Client configuration for specifying connection options. See
-   *        {@link ClientConfiguration} which extends Configuration with convenience methods
-   *        specific to Accumulo.
-   * @since 1.9.0
-   */
-  public ZooKeeperInstance(ClientConfiguration config) {
-    this(config, new ZooCacheFactory());
-  }
-
-  @Override
-  public String getInstanceID() {
-    if (instanceId == null) {
-      instanceId = ClientContext.getInstanceID(zooCache, instanceName).canonical();
-    }
-    ClientContext.verifyInstanceId(zooCache, instanceId, instanceName);
-    return instanceId;
-  }
-
-  @Override
-  public List<String> getMasterLocations() {
-    return ClientContext.getManagerLocations(zooCache, getInstanceID());
-  }
-
-  @Override
-  public String getRootTabletLocation() {
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up root tablet location in zookeeper.",
-          Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    Location loc = TabletsMetadata
-        .getRootMetadata(Constants.ZROOT + "/" + getInstanceID(), zooCache).getLocation();
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(), loc,
-          String.format("%.3f secs", timer.scale(SECONDS)));
-    }
-
-    if (loc == null || loc.getType() != LocationType.CURRENT) {
-      return null;
-    }
-
-    return loc.getHostPort();
-  }
-
-  @Override
-  public String getInstanceName() {
-    if (instanceName == null) {
-      instanceName =
-          InstanceOperationsImpl.lookupInstanceName(zooCache, InstanceId.of(getInstanceID()));
-    }
-
-    return instanceName;
-  }
-
-  @Override
-  public String getZooKeepers() {
-    return zooKeepers;
-  }
-
-  @Override
-  public int getZooKeepersSessionTimeOut() {
-    return zooKeepersSessionTimeOut;
-  }
-
-  @Override
-  public Connector getConnector(String principal, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException {
-    Properties properties = ClientConfConverter.toProperties(clientConf);
-    properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
-    properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), getInstanceName());
-    ClientInfo info = new ClientInfoImpl(properties, token);
-    AccumuloConfiguration serverConf = ClientConfConverter.toAccumuloConf(properties);
-    return new org.apache.accumulo.core.clientImpl.ConnectorImpl(
-        new ClientContext(SingletonReservation.noop(), info, serverConf, null));
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(64);
-    sb.append("ZooKeeperInstance: ").append(getInstanceName()).append(" ").append(getZooKeepers());
-    return sb.toString();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index 45a31e9..dd02f6b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -29,17 +29,12 @@
  */
 public abstract class ActiveCompaction {
 
-  public static enum CompactionType {
+  public enum CompactionType {
     /**
      * compaction to flush a tablets memory
      */
     MINOR,
     /**
-     * Accumulo no longer does merging minor compactions.
-     */
-    @Deprecated(since = "2.1.0", forRemoval = true)
-    MERGE,
-    /**
      * compaction that merges a subset of a tablets files into one file
      */
     MAJOR,
@@ -49,7 +44,7 @@
     FULL
   }
 
-  public static enum CompactionReason {
+  public enum CompactionReason {
     /**
      * compaction initiated by user
      */
@@ -59,8 +54,10 @@
      */
     SYSTEM,
     /**
-     * Compaction initiated by merge operation
+     * @deprecated Chop compactions no longer occur and it's not expected that listing compaction
+     *             would ever return this.
      */
+    @Deprecated(since = "3.1")
     CHOP,
     /**
      * idle compaction
@@ -125,6 +122,12 @@
   public abstract long getEntriesWritten();
 
   /**
+   * @return the number of times the server paused a compaction
+   * @since 3.0.0
+   */
+  public abstract long getPausedCount();
+
+  /**
    * @return the per compaction iterators configured
    */
   public abstract List<IteratorSetting> getIterators();
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
index 64392a1..1600ea0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionConfig.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.client.admin;
 
 import static java.util.Objects.requireNonNull;
-import static org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil.DEFAULT_STRATEGY;
 
 import java.util.Collections;
 import java.util.List;
@@ -29,7 +28,6 @@
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.admin.compaction.CompactionConfigurer;
 import org.apache.accumulo.core.client.admin.compaction.CompactionSelector;
-import org.apache.accumulo.core.clientImpl.CompactionStrategyConfigUtil;
 import org.apache.accumulo.core.clientImpl.UserCompactionUtils;
 import org.apache.hadoop.io.Text;
 
@@ -40,7 +38,6 @@
  *
  * @since 1.7.0
  */
-@SuppressWarnings("removal")
 public class CompactionConfig {
 
   private Text start = null;
@@ -48,7 +45,6 @@
   private boolean flush = true;
   private boolean wait = true;
   private List<IteratorSetting> iterators = Collections.emptyList();
-  private CompactionStrategyConfig compactionStrategy = DEFAULT_STRATEGY;
   private Map<String,String> hints = Map.of();
   private PluginConfig selectorConfig = UserCompactionUtils.DEFAULT_SELECTOR;
   private PluginConfig configurerConfig = UserCompactionUtils.DEFAULT_CONFIGURER;
@@ -144,35 +140,6 @@
   }
 
   /**
-   * @param csConfig configures the strategy that will be used by each tablet to select files. If no
-   *        strategy is set, then all files will be compacted.
-   * @return this
-   * @deprecated since 2.1.0 use {@link #setSelector(PluginConfig)} and
-   *             {@link #setConfigurer(PluginConfig)} instead. See {@link CompactionStrategyConfig}
-   *             for details about why this was deprecated.
-   */
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public CompactionConfig setCompactionStrategy(CompactionStrategyConfig csConfig) {
-    requireNonNull(csConfig);
-    Preconditions.checkArgument(!csConfig.getClassName().isBlank());
-    Preconditions.checkState(
-        selectorConfig.getClassName().isEmpty() && configurerConfig.getClassName().isEmpty());
-    this.compactionStrategy = csConfig;
-    return this;
-  }
-
-  /**
-   * @return The previously set compaction strategy. Defaults to a configuration of
-   *         org.apache.accumulo.tserver.compaction.EverythingCompactionStrategy which always
-   *         compacts all files.
-   * @deprecated since 2.1.0
-   */
-  @Deprecated(since = "2.1.0", forRemoval = true)
-  public CompactionStrategyConfig getCompactionStrategy() {
-    return compactionStrategy;
-  }
-
-  /**
    * Configure a {@link CompactionSelector} plugin to run for this compaction. Specify the class
    * name and options here.
    *
@@ -180,7 +147,6 @@
    * @since 2.1.0
    */
   public CompactionConfig setSelector(PluginConfig selectorConfig) {
-    Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
     Preconditions.checkArgument(!selectorConfig.getClassName().isBlank());
     this.selectorConfig = requireNonNull(selectorConfig);
     return this;
@@ -197,9 +163,6 @@
    * @since 2.1.0
    */
   public CompactionConfig setExecutionHints(Map<String,String> hints) {
-    if (!hints.isEmpty()) {
-      Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
-    }
     this.hints = Map.copyOf(hints);
     return this;
   }
@@ -218,7 +181,6 @@
    * @since 2.1.0
    */
   public CompactionConfig setConfigurer(PluginConfig configurerConfig) {
-    Preconditions.checkState(compactionStrategy.getClassName().isEmpty());
     this.configurerConfig = configurerConfig;
     return this;
   }
@@ -251,8 +213,6 @@
     prefix = append(sb, prefix, () -> !flush, "flush", flush);
     prefix = append(sb, prefix, () -> !wait, "wait", wait);
     prefix = append(sb, prefix, () -> !iterators.isEmpty(), "iterators", iterators);
-    prefix = append(sb, prefix, () -> !CompactionStrategyConfigUtil.isDefault(compactionStrategy),
-        "strategy", compactionStrategy);
     prefix = append(sb, prefix, () -> !UserCompactionUtils.isDefault(selectorConfig), "selector",
         selectorConfig);
     prefix = append(sb, prefix, () -> !UserCompactionUtils.isDefault(configurerConfig),
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java b/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java
deleted file mode 100644
index 2f880a6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/CompactionStrategyConfig.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.admin;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Map;
-
-/**
- * Configuration object which describes how a Compaction is run. Configuration objects are dependent
- * upon the CompactionStrategy running insider the server. This class is used in conjunction with
- * {@link CompactionConfig}.
- *
- * @since 1.7.0
- * @deprecated since 2.1.0 CompactionStrategies were deprecated for multiple reasons. First, they do
- *             not support the new compaction execution model. Second, they bind selection and
- *             output file configuration into a single entity when users need to configure these
- *             independently. Third, they use internal Accumulo types and ensuring their stability
- *             requires manual effort that may never happen. Fourth, writing a correct compaction
- *             strategy was exceedingly difficult as it required knowledge of internal tablet server
- *             synchronization in order to avoid causing scans to hang. Fifth although measure were
- *             taken to execute compaction strategies in the same manner as before, their execution
- *             in the new model has subtle differences that may result in suboptimal compactions.
- *             Please migrate to using {@link CompactionConfig#setSelector(PluginConfig)} and
- *             {@link CompactionConfig#setConfigurer(PluginConfig)} as soon as possible.
- */
-@Deprecated(since = "2.1.0", forRemoval = true)
-public class CompactionStrategyConfig {
-  private String className;
-  private Map<String,String> options = Map.of();
-
-  /**
-   * @param className The name of a class that implements
-   *        org.apache.accumulo.tserver.compaction.CompactionStrategy. This class must exist on
-   *        tservers.
-   */
-  public CompactionStrategyConfig(String className) {
-    this.className = requireNonNull(className);
-  }
-
-  /**
-   * @return the class name passed to the constructor.
-   */
-  public String getClassName() {
-    return className;
-  }
-
-  /**
-   * @param opts The options that will be passed to the init() method of the compaction strategy
-   *        when its instantiated on a tserver. This method will copy the map. The default is an
-   *        empty map.
-   * @return this
-   */
-  public CompactionStrategyConfig setOptions(Map<String,String> opts) {
-    this.options = Map.copyOf(opts);
-    return this;
-  }
-
-  /**
-   * @return The previously set options. Returns an unmodifiable map. The default is an empty map.
-   */
-  public Map<String,String> getOptions() {
-    return options;
-  }
-
-  @Override
-  public int hashCode() {
-    return className.hashCode() + options.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof CompactionStrategyConfig) {
-      CompactionStrategyConfig ocsc = (CompactionStrategyConfig) o;
-      return className.equals(ocsc.className) && options.equals(ocsc.options);
-    }
-
-    return false;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java b/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
index 2caba85..a953d7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/FindMax.java
@@ -67,7 +67,7 @@
 
     if (ba.length == startOS.size()) {
       if (ba[0] != 0) {
-        throw new RuntimeException();
+        throw new IllegalStateException();
       }
 
       // big int added a zero so it would not be negative, drop it
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
index 2e1c0c9..4cb3f6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
@@ -87,22 +87,22 @@
    *         {@code
    *             AccumuloClient client = getClient();
    *             Map<String,String> acceptedProps = client.instanceOperations().modifyProperties(currProps -> {
-   *               var planner = currProps.get("tserver.compaction.major.service.default.planner");
+   *               var planner = currProps.get("compaction.service.default.planner");
    *               //This code will only change the compaction planner if its currently set to default settings.
    *               //The endsWith() function was used to make the example short, would be better to use equals().
    *               if(planner != null && planner.endsWith("DefaultCompactionPlanner") {
    *                 // tservers will eventually see these compaction planner changes and when they do they will see all of the changes at once
    *                 currProps.keySet().removeIf(
-   *                    prop -> prop.startsWith("tserver.compaction.major.service.default.planner.opts."));
-   *                 currProps.put("tserver.compaction.major.service.default.planner","MyPlannerClassName");
-   *                 currProps.put("tserver.compaction.major.service.default.planner.opts.myOpt1","val1");
-   *                 currProps.put("tserver.compaction.major.service.default.planner.opts.myOpt2","val2");
+   *                    prop -> prop.startsWith("compaction.service.default.planner.opts."));
+   *                 currProps.put("compaction.service.default.planner","MyPlannerClassName");
+   *                 currProps.put("compaction.service.default.planner.opts.myOpt1","val1");
+   *                 currProps.put("compaction.service.default.planner.opts.myOpt2","val2");
    *                }
    *             });
    *
    *             // Since three properties were set may want to check for the values of all
    *             // three, just checking one in this example to keep it short.
-   *             if("MyPlannerClassName".equals(acceptedProps.get("tserver.compaction.major.service.default.planner"))){
+   *             if("MyPlannerClassName".equals(acceptedProps.get("compaction.service.default.planner"))){
    *                // the compaction planner change was accepted or already existed, so take action for that outcome
    *             } else {
    *                // the compaction planner change was not done, so take action for that outcome
@@ -140,7 +140,17 @@
   void removeProperty(final String property) throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * Retrieve the system-wide configuration.
+   * Retrieve the system-wide, merged view of the system configuration. Accumulo has multiple layers
+   * of properties, in order of precedence (highest - lowest):
+   * <ul>
+   * <li>the properties set in Zookeeper</li>
+   * <li>the properties set in the site configuration file</li>
+   * <li>the default properties</li>
+   * </ul>
+   * The properties returned is the merged view of these properties. The properties that are stored
+   * in ZooKeeper can be modified with {@link #modifyProperties modifyProperties},
+   * {@link #setProperty setProperty} and {@link #removeProperty removeProperty}. Properties can be
+   * further refined by namesapce {@link NamespaceOperations} and by table {@link TableOperations}.
    *
    * @return A map of system properties set in zookeeper. If a property is not set in zookeeper,
    *         then it will return the value set in accumulo.properties on some server. If nothing is
@@ -157,6 +167,16 @@
   Map<String,String> getSiteConfiguration() throws AccumuloException, AccumuloSecurityException;
 
   /**
+   * Retrieve a map of the system properties set in Zookeeper. Note that this does not return a
+   * merged view of the properties from its parent configuration. See
+   * {@link #getSystemConfiguration} for a merged view.
+   *
+   * @return A map of the system properties set in Zookeeper only.
+   * @since 3.1
+   */
+  Map<String,String> getSystemProperties() throws AccumuloException, AccumuloSecurityException;
+
+  /**
    * Returns the location(s) of the accumulo manager and any redundant servers.
    *
    * @return a list of locations in <code>hostname:port</code> form.
@@ -237,17 +257,6 @@
   void waitForBalance() throws AccumuloException;
 
   /**
-   * Returns a unique string that identifies this instance of accumulo.
-   *
-   * @return a String
-   * @since 2.0.0
-   *
-   * @deprecated in 2.1.0 Use {@link #getInstanceId()}
-   */
-  @Deprecated(since = "2.1.0")
-  String getInstanceID();
-
-  /**
    * Returns a unique ID object that identifies this instance of accumulo.
    *
    * @return an InstanceId
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
deleted file mode 100644
index 2815174..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.admin;
-
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.replication.PeerExistsException;
-import org.apache.accumulo.core.client.replication.PeerNotFoundException;
-
-/**
- * Supports replication configuration
- *
- * @since 1.7.0
- */
-@Deprecated
-public interface ReplicationOperations {
-
-  /**
-   * Defines a cluster with the given name and the given name system.
-   *
-   * @param name Unique name for the cluster
-   * @param replicaType Class name to use to replicate the data
-   */
-  void addPeer(String name, String replicaType)
-      throws AccumuloException, AccumuloSecurityException, PeerExistsException;
-
-  /**
-   * Removes a cluster with the given name.
-   *
-   * @param name Name of the cluster to remove
-   */
-  void removePeer(String name)
-      throws AccumuloException, AccumuloSecurityException, PeerNotFoundException;
-
-  /**
-   * Waits for a table to be fully replicated, given the state of files pending replication for the
-   * provided table at the point in time which this method is invoked.
-   *
-   * @param tableName The table to wait for
-   */
-  void drain(String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-
-  /**
-   * Given the provided set of files that are pending replication for a table, wait for those files
-   * to be fully replicated to all configured peers. This allows for the accurate calculation when a
-   * table, at a given point in time, has been fully replicated.
-   *
-   * @param tableName The table to wait for
-   */
-  void drain(String tableName, Set<String> files)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-
-  /**
-   * Gets all of the referenced files for a table from the metadata table. The result of this method
-   * is intended to be directly supplied to {@link #drain(String, Set)}. This helps determine when
-   * all data from a given point in time has been fully replicated.
-   * <p>
-   * This also allows callers to get the {@link Set} of files for a table at some time, and later
-   * provide that {@link Set} to {@link #drain(String,Set)} to wait for all of those files to be
-   * replicated.
-   */
-  Set<String> referencedFiles(String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index eabafaa..fef5599 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -88,47 +88,6 @@
       throws AccumuloException, AccumuloSecurityException, TableExistsException;
 
   /**
-   * @param tableName the name of the table
-   * @param limitVersion Enables/disables the versioning iterator, which will limit the number of
-   *        Key versions kept.
-   * @throws AccumuloException if a general error occurs
-   * @throws AccumuloSecurityException if the user does not have permission
-   * @throws TableExistsException if the table already exists
-   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
-   */
-  @Deprecated(since = "1.7.0")
-  default void create(String tableName, boolean limitVersion)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    if (limitVersion) {
-      create(tableName);
-    } else {
-      create(tableName, new NewTableConfiguration().withoutDefaultIterators());
-    }
-  }
-
-  /**
-   * @param tableName the name of the table
-   * @param versioningIter Enables/disables the versioning iterator, which will limit the number of
-   *        Key versions kept.
-   * @param timeType specifies logical or real-time based time recording for entries in the table
-   * @throws AccumuloException if a general error occurs
-   * @throws AccumuloSecurityException if the user does not have permission
-   * @throws TableExistsException if the table already exists
-   * @deprecated since 1.7.0; use {@link #create(String, NewTableConfiguration)} instead.
-   */
-  @Deprecated(since = "1.7.0")
-  default void create(String tableName, boolean versioningIter, TimeType timeType)
-      throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    NewTableConfiguration ntc = new NewTableConfiguration().setTimeType(timeType);
-
-    if (versioningIter) {
-      create(tableName, ntc);
-    } else {
-      create(tableName, ntc.withoutDefaultIterators());
-    }
-  }
-
-  /**
    * Create a table with specified configuration. A safe way to ignore tables that do exist would be
    * to do something like the following:
    *
@@ -228,21 +187,6 @@
    * @param tableName the name of the table
    * @return the split points (end-row names) for the table's current split profile
    * @throws TableNotFoundException if the table does not exist
-   * @deprecated since 1.5.0; use {@link #listSplits(String)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  default Collection<Text> getSplits(String tableName) throws TableNotFoundException {
-    try {
-      return listSplits(tableName);
-    } catch (AccumuloSecurityException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * @param tableName the name of the table
-   * @return the split points (end-row names) for the table's current split profile
-   * @throws TableNotFoundException if the table does not exist
    * @throws AccumuloException if a general error occurs
    * @throws AccumuloSecurityException if the user does not have permission
    * @since 1.5.0
@@ -253,23 +197,6 @@
   /**
    * @param tableName the name of the table
    * @param maxSplits specifies the maximum number of splits to return
-   * @return the split points (end-row names) for the table's current split profile, grouped into
-   *         fewer splits so as not to exceed maxSplits
-   * @deprecated since 1.5.0; use {@link #listSplits(String, int)} instead.
-   */
-  @Deprecated(since = "1.5.0")
-  default Collection<Text> getSplits(String tableName, int maxSplits)
-      throws TableNotFoundException {
-    try {
-      return listSplits(tableName, maxSplits);
-    } catch (AccumuloSecurityException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * @param tableName the name of the table
-   * @param maxSplits specifies the maximum number of splits to return
    * @throws AccumuloException if a general error occurs
    * @throws AccumuloSecurityException if the user does not have permission
    * @return the split points (end-row names) for the table's current split profile, grouped into
@@ -363,8 +290,8 @@
 
   /**
    * Starts a full major compaction of the tablets in the range (start, end]. If the config does not
-   * specify a compaction selector (or a deprecated strategy), then all files in a tablet are
-   * compacted. The compaction is performed even for tablets that have only one file.
+   * specify a compaction selector, then all files in a tablet are compacted. The compaction is
+   * performed even for tablets that have only one file.
    *
    * <p>
    * The following optional settings can only be set by one compact call per table at the same time.
@@ -374,8 +301,6 @@
    * <li>Selector : {@link CompactionConfig#setSelector(PluginConfig)}</li>
    * <li>Configurer : {@link CompactionConfig#setConfigurer(PluginConfig)}</li>
    * <li>Iterators : {@link CompactionConfig#setIterators(List)}</li>
-   * <li>Compaction strategy (deprecated) :
-   * {@code CompactionConfig.setCompactionStrategy(CompactionStrategyConfig)}</li>
    * </ul>
    *
    * <p>
@@ -640,27 +565,6 @@
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
 
   /**
-   * Bulk import all the files in a directory into a table. Files can be created using
-   * {@link RFile#newWriter()}
-   *
-   * @param tableName the name of the table
-   * @param dir the HDFS directory to find files for importing
-   * @param failureDir the HDFS directory to place files that failed to be imported, must exist and
-   *        be empty
-   * @param setTime override the time values in the input files, and use the current time for all
-   *        mutations
-   * @throws IOException when there is an error reading/writing to HDFS
-   * @throws AccumuloException when there is a general accumulo error
-   * @throws AccumuloSecurityException when the user does not have the proper permissions
-   * @throws TableNotFoundException when the table no longer exists
-   *
-   * @deprecated since 2.0.0 use {@link #importDirectory(String)} instead.
-   */
-  @Deprecated(since = "2.0.0")
-  void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
-      throws TableNotFoundException, IOException, AccumuloException, AccumuloSecurityException;
-
-  /**
    * @since 2.0.0
    */
   interface ImportOptions {
@@ -749,11 +653,9 @@
    * Bulk import the files in a directory into a table. Files can be created using
    * {@link RFile#newWriter()}.
    * <p>
-   * This new method of bulk import examines files in the current process outside of holding a table
-   * lock. The old bulk import method ({@link #importDirectory(String, String, String, boolean)})
-   * examines files on the server side while holding a table read lock.
-   * <p>
-   * This API supports adding files to online and offline tables.
+   * This API supports adding files to online and offline tables. The files are examined on the
+   * client side to determine destination tablets. This examination will use memory and cpu within
+   * the process calling this API.
    * <p>
    * For example, to bulk import files from the directory 'dir1' into the table 'table1' use the
    * following code.
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
index baf5f04..36b2d7b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactableFile.java
@@ -20,6 +20,7 @@
 
 import java.net.URI;
 
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.metadata.CompactableFileImpl;
 
 /**
@@ -33,6 +34,16 @@
 
   public URI getUri();
 
+  /**
+   * @return A range associated with the file. If a file has an associated range then Accumulo will
+   *         limit reads to within the range. Not all files have an associated range, it a file does
+   *         not have a range then an infinite range is returned. The URI plus this range uniquely
+   *         identify a file.
+   *
+   * @since 3.1.0
+   */
+  public Range getRange();
+
   public long getEstimatedSize();
 
   public long getEstimatedEntries();
@@ -41,4 +52,12 @@
     return new CompactableFileImpl(uri, estimatedSize, estimatedEntries);
   }
 
+  /**
+   * Creates a new CompactableFile object that implements this interface.
+   *
+   * @since 3.1.0
+   */
+  static CompactableFile create(URI uri, Range range, long estimatedSize, long estimatedEntries) {
+    return new CompactableFileImpl(uri, range, estimatedSize, estimatedEntries);
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
index 6a58704..1e0e47c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionConfigurer.java
@@ -20,9 +20,11 @@
 
 import java.util.Collection;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.accumulo.core.client.PluginEnvironment;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
 
 /**
  * Enables dynamically overriding of per table properties used to create the output file for a
@@ -50,8 +52,36 @@
   public interface InputParameters {
     TableId getTableId();
 
+    /**
+     * @return the id of the tablet being compacted
+     * @since 3.0.0
+     */
+    TabletId getTabletId();
+
     public Collection<CompactableFile> getInputFiles();
 
+    /**
+     * For user and selector compactions:
+     * <ul>
+     * <li>Returns the selected set of files to be compacted.</li>
+     * <li>When getInputFiles() (inputFiles) and getSelectedFiles() (selectedFiles) are equal, then
+     * this is the final compaction.</li>
+     * <li>When they are not equal, this is an intermediate compaction.</li>
+     * <li>Intermediate compactions are compactions whose resultant RFile will be consumed by
+     * another compaction.</li>
+     * <li>inputFiles and selectedFiles can be compared using: <code>
+     * selectedFiles.equals(inputFiles instanceof Set ? inputFiles : Set.copyOf(inputFiles))
+     * </code></li>
+     * </ul>
+     * For system compactions:
+     * <ul>
+     * <li>There is no selected set of files so the empty set is returned.</li>
+     * </ul>
+     *
+     * @since 3.1
+     */
+    public Set<CompactableFile> getSelectedFiles();
+
     PluginEnvironment getEnvironment();
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
index d54f612..713a451 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/compaction/CompactionSelector.java
@@ -30,12 +30,14 @@
 import org.apache.accumulo.core.client.summary.Summary;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
 /**
  * This class selects which files a user compaction will compact. It can also be configured per
- * table to periodically select files to compact.
+ * table to periodically select files to compact, although per table functionality is deprecated.
+ * See {@link org.apache.accumulo.core.spi.compaction.CompactionKind#SELECTOR} for details.
  *
  * @since 2.1.0
  */
@@ -61,6 +63,12 @@
 
     TableId getTableId();
 
+    /**
+     * @return the tablet id of the tablet being compacted
+     * @since 3.0.0
+     */
+    TabletId getTabletId();
+
     Optional<SortedKeyValueIterator<Key,Value>> getSample(CompactableFile cf,
         SamplerConfiguration sc);
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
index 1d468e4..9195684 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/BigIntegerLexicoder.java
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.math.BigInteger;
 
 import org.apache.accumulo.core.clientImpl.lexicoder.FixedByteArrayOutputStream;
@@ -59,7 +60,7 @@
 
       return ret;
     } catch (IOException ioe) {
-      throw new RuntimeException(ioe);
+      throw new UncheckedIOException(ioe);
     }
 
   }
@@ -87,7 +88,7 @@
 
       return new BigInteger(bytes);
     } catch (IOException ioe) {
-      throw new RuntimeException(ioe);
+      throw new UncheckedIOException(ioe);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
index 590c370..e5172a5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java
@@ -80,7 +80,7 @@
 
     byte[][] fields = split(data, offset, len);
     if (fields.length != 2) {
-      throw new RuntimeException("Data does not have 2 fields, it has " + fields.length);
+      throw new IllegalArgumentException("Data does not have 2 fields, it has " + fields.length);
     }
 
     return new ComparablePair<>(firstLexicoder.decode(unescape(fields[0])),
diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
index b3a7bc2..fa1f3fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/UUIDLexicoder.java
@@ -22,6 +22,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.UUID;
 
 import org.apache.accumulo.core.clientImpl.lexicoder.FixedByteArrayOutputStream;
@@ -52,7 +53,7 @@
 
       return ret;
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -69,7 +70,7 @@
       DataInputStream in = new DataInputStream(new ByteArrayInputStream(b, offset, len));
       return new UUID(in.readLong() ^ 0x8000000000000000L, in.readLong() ^ 0x8000000000000000L);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
deleted file mode 100644
index 088a065..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ /dev/null
@@ -1,753 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At
- * the very least, any classes inheriting from this class will need to define their own
- * {@link RecordReader}.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
-
-  private static final SecureRandom random = new SecureRandom();
-
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the name of the classloader context on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param context name of the classloader context
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(JobConf job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job, context);
-  }
-
-  /**
-   * Returns the name of the current classloader context set on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return name of the current context
-   * @since 1.8.0
-   */
-  public static String getClassLoaderContext(JobConf job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = InputConfigurator.client(CLASS, job);
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, Mappers/Reducers will likely"
-            + " fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param tokenFile the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobConf job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobConf job) {
-    return InputConfigurator.getPrincipal(CLASS, job);
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   * @see #setConnectorInfo(JobConf, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    AuthenticationToken token = InputConfigurator.getAuthenticationToken(CLASS, job);
-    return InputConfigurator.unwrapAuthenticationToken(job, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use
-   *             {@link #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
-   *             instead.
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param clientConfig client configuration containing connection options
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(JobConf job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
-    return InputConfigurator.getInstance(CLASS, job);
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(JobConf job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(JobConf, Level)
-   */
-  protected static Level getLogLevel(JobConf job) {
-    return InputConfigurator.getLogLevel(CLASS, job);
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization.
-   * Defaults to the empty set.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param auths the user's authorizations
-   * @since 1.5.0
-   */
-  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(JobConf, Authorizations)
-   */
-  protected static Authorizations getScanAuthorizations(JobConf job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job);
-  }
-
-  /**
-   * Fetch the client configuration from the job.
-   *
-   * @param job The job
-   * @return The client configuration for the job
-   * @since 1.7.0
-   */
-  protected static org.apache.accumulo.core.client.ClientConfiguration
-      getClientConfiguration(JobConf job) {
-    return InputConfigurator.getClientConfiguration(CLASS, job);
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo
-   * {@link InputFormat}.
-   *
-   * @param job the Hadoop context for the configured job
-   * @throws java.io.IOException if the context is improperly configured
-   * @since 1.5.0
-   */
-  protected static void validateOptions(JobConf job) throws IOException {
-    InputConfigurator.validatePermissions(CLASS, job);
-  }
-
-  /**
-   * Fetches all {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig}s that have been
-   * set on the given Hadoop job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects set on
-   *         the job
-   * @since 1.6.0
-   */
-  public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig>
-      getInputTableConfigs(JobConf job) {
-    return InputConfigurator.getInputTableConfigs(CLASS, job);
-  }
-
-  /**
-   * Fetches a {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} that has been set
-   * on the configuration for a specific table.
-   *
-   * <p>
-   * null is returned in the event that the table doesn't exist.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table name for which to grab the config object
-   * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the given
-   *         table
-   * @since 1.6.0
-   */
-  public static org.apache.accumulo.core.client.mapreduce.InputTableConfig
-      getInputTableConfig(JobConf job, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, job, tableName);
-  }
-
-  /**
-   * An abstract base class to be used to create {@link RecordReader} instances that convert from
-   * Accumulo {@link Key}/{@link Value} pairs to the user's K/V types.
-   *
-   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to
-   * update the following variables:
-   * <ul>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  protected abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
-    protected long numKeysRead;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected RangeInputSplit split;
-    private org.apache.accumulo.core.client.mapreduce.RangeInputSplit baseSplit;
-    protected ScannerBase scannerBase;
-
-    /**
-     * Extracts Iterators settings from the context to be used by RecordReader.
-     *
-     * @param job the Hadoop job configuration
-     * @param tableName the table name for which the scanner is configured
-     * @return List of iterator settings for given table
-     * @since 1.7.0
-     */
-    protected abstract List<IteratorSetting> jobIterators(JobConf job, String tableName);
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param job the Hadoop job configuration
-     * @param scanner the scanner for which to configure the iterators
-     * @param tableName the table name for which the scanner is configured
-     * @since 1.7.0
-     */
-    private void setupIterators(JobConf job, ScannerBase scanner, String tableName,
-        org.apache.accumulo.core.client.mapreduce.RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (split == null) {
-        iterators = jobIterators(job, tableName);
-      } else {
-        iterators = split.getIterators();
-        if (iterators == null) {
-          iterators = jobIterators(job, tableName);
-        }
-      }
-
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param job the Hadoop job configuration
-     * @param scanner the scanner for which to configure the iterators
-     * @param tableName the table name for which the scanner is configured
-     * @since 1.6.0
-     * @deprecated since 1.7.0; Use {@link #jobIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(JobConf job, Scanner scanner, String tableName,
-        RangeInputSplit split) {
-      setupIterators(job, (ScannerBase) scanner, tableName, split);
-    }
-
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     */
-    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-      baseSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + baseSplit);
-
-      Authorizations authorizations = baseSplit.getAuths();
-      if (null == authorizations) {
-        authorizations = getScanAuthorizations(job);
-      }
-
-      String classLoaderContext = getClassLoaderContext(job);
-      String table = baseSplit.getTableName();
-
-      // in case the table name changed, we can still use the previous name for terms of
-      // configuration, but the scanner will use the table id resolved at job setup time
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
-          getInputTableConfig(job, baseSplit.getTableName());
-
-      ClientContext client = InputConfigurator.client(CLASS, baseSplit, job);
-
-      log.debug("Created client with user: " + client.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (baseSplit instanceof org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit) {
-        BatchScanner scanner;
-        org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit multiRangeSplit =
-            (org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit) baseSplit;
-
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner =
-              client.createBatchScanner(baseSplit.getTableName(), authorizations, scanThreads);
-          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-          if (classLoaderContext != null) {
-            scanner.setClassLoaderContext(classLoaderContext);
-          }
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRanges(multiRangeSplit.getRanges());
-        scannerBase = scanner;
-
-      } else if (baseSplit instanceof RangeInputSplit) {
-        split = (RangeInputSplit) baseSplit;
-        Boolean isOffline = baseSplit.isOffline();
-        if (isOffline == null) {
-          isOffline = tableConfig.isOfflineScan();
-        }
-
-        Boolean isIsolated = baseSplit.isIsolatedScan();
-        if (isIsolated == null) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
-
-        Boolean usesLocalIterators = baseSplit.usesLocalIterators();
-        if (usesLocalIterators == null) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
-
-        Scanner scanner;
-
-        try {
-          if (isOffline) {
-            scanner =
-                new OfflineScanner(client, TableId.of(baseSplit.getTableId()), authorizations);
-          } else {
-            scanner = new ScannerImpl(client, TableId.of(baseSplit.getTableId()), authorizations);
-          }
-          if (isIsolated) {
-            log.info("Creating isolated scanner");
-            @SuppressWarnings("resource")
-            var wrapped = new IsolatedScanner(scanner);
-            scanner = wrapped;
-          }
-          if (usesLocalIterators) {
-            log.info("Using local iterators");
-            scanner = new ClientSideIteratorScanner(scanner);
-          }
-          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRange(baseSplit.getRange());
-        scannerBase = scanner;
-      } else {
-        throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
-      }
-
-      Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
-      if (columns == null) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : columns) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
-      if (samplerConfig == null) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
-
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
-    }
-
-    @Override
-    public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
-    }
-
-    @Override
-    public long getPos() throws IOException {
-      return numKeysRead;
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      if (numKeysRead > 0 && currentKey == null) {
-        return 1.0f;
-      }
-      return baseSplit.getProgress(currentKey);
-    }
-
-    protected Key currentKey = null;
-
-  }
-
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, TableId tableId,
-      List<Range> ranges)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    return InputConfigurator.binOffline(tableId, ranges, InputConfigurator.client(CLASS, job));
-  }
-
-  /**
-   * Gets the splits of the tables that have been set on the job by reading the metadata table for
-   * the specified ranges.
-   *
-   * @return the splits from the tables based on the ranges.
-   * @throws java.io.IOException if a table set on the job doesn't exist or an error occurs
-   *         initializing the tablet locator
-   */
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    Level logLevel = getLogLevel(job);
-    log.setLevel(logLevel);
-    validateOptions(job);
-
-    LinkedList<InputSplit> splits = new LinkedList<>();
-    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigs =
-        getInputTableConfigs(job);
-
-    for (Map.Entry<String,
-        org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigEntry : tableConfigs
-            .entrySet()) {
-
-      String tableName = tableConfigEntry.getKey();
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig =
-          tableConfigEntry.getValue();
-
-      ClientContext client;
-      try {
-        client = InputConfigurator.client(CLASS, job);
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IOException(e);
-      }
-
-      TableId tableId;
-      // resolve table name to id once, and use id from this point forward
-      try {
-        tableId = client.getTableId(tableName);
-      } catch (TableNotFoundException e) {
-        throw new IOException(e);
-      }
-
-      boolean batchScan = InputConfigurator.isBatchScan(CLASS, job);
-      boolean supportBatchScan = !(tableConfig.isOfflineScan()
-          || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
-      if (batchScan && !supportBatchScan) {
-        throw new IllegalArgumentException("BatchScanner optimization not available for offline"
-            + " scan, isolated, or local iterators");
-      }
-
-      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-      if (batchScan && !autoAdjust) {
-        throw new IllegalArgumentException(
-            "AutoAdjustRanges must be enabled when using BatchScanner optimization");
-      }
-
-      List<Range> ranges =
-          autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
-      if (ranges.isEmpty()) {
-        ranges = new ArrayList<>(1);
-        ranges.add(new Range());
-      }
-
-      // get the metadata information for these ranges
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-      TabletLocator tl;
-      try {
-        if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(job, tableId, ranges);
-          while (binnedRanges == null) {
-            // Some tablets were still online, try again
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            binnedRanges = binOfflineTable(job, tableId, ranges);
-          }
-        } else {
-          tl = TabletLocator.getLocator(client, tableId);
-          // its possible that the cache could contain complete, but old information about a
-          // tables tablets... so clear it
-          tl.invalidateCache();
-
-          while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
-            client.requireNotDeleted(tableId);
-            client.requireNotOffline(tableId, tableName);
-            binnedRanges.clear();
-            log.warn("Unable to locate bins for specified ranges. Retrying.");
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            tl.invalidateCache();
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      // all of this code will add either range per each locations or split ranges and add
-      // range-location split
-      // Map from Range to Array of Locations, we only use this if we're don't split
-      HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-      if (!autoAdjust) {
-        splitsToAdd = new HashMap<>();
-      }
-
-      HashMap<String,String> hostNameCache = new HashMap<>();
-      for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
-        String ip = tserverBin.getKey().split(":", 2)[0];
-        String location = hostNameCache.get(ip);
-        if (location == null) {
-          InetAddress inetAddress = InetAddress.getByName(ip);
-          location = inetAddress.getCanonicalHostName();
-          hostNameCache.put(ip, location);
-        }
-        for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-          Range ke = extentRanges.getKey().toDataRange();
-          if (batchScan) {
-            // group ranges by tablet to be read by a BatchScanner
-            ArrayList<Range> clippedRanges = new ArrayList<>();
-            for (Range r : extentRanges.getValue()) {
-              clippedRanges.add(ke.clip(r));
-            }
-            org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit split =
-                new org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit(tableName, tableId,
-                    clippedRanges, new String[] {location});
-            org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-                logLevel);
-
-            splits.add(split);
-          } else {
-            // not grouping by tablet
-            for (Range r : extentRanges.getValue()) {
-              if (autoAdjust) {
-                // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-                    ke.clip(r), new String[] {location});
-                org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
-                    tableConfig, logLevel);
-                split.setOffline(tableConfig.isOfflineScan());
-                split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-                splits.add(split);
-              } else {
-                // don't divide ranges
-                ArrayList<String> locations = splitsToAdd.get(r);
-                if (locations == null) {
-                  locations = new ArrayList<>(1);
-                }
-                locations.add(location);
-                splitsToAdd.put(r, locations);
-              }
-            }
-          }
-        }
-      }
-
-      if (!autoAdjust) {
-        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-              entry.getKey(), entry.getValue().toArray(new String[0]));
-          org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-              logLevel);
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-          splits.add(split);
-        }
-      }
-    }
-    return splits.toArray(new InputSplit[splits.size()]);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
deleted file mode 100644
index 75c11ad..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.rfile.RFile;
-import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br>
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important
- * requirement of Accumulo data files.
- *
- * <p>
- * The output path to be created must be specified via
- * {@link AccumuloFileOutputFormat#setOutputPath(JobConf, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Other methods from
- * {@link FileOutputFormat} are not supported and may be ignored or cause failures. Using other
- * Hadoop configuration options that affect the behavior of the underlying files directly in the
- * Job's configuration may work, but are not directly supported at this time.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require
-   * additional libraries to be available to your Job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
-   * @since 1.5.0
-   */
-  public static void setCompressionType(JobConf job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job, compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
-   * a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size
-   * of the indexes (which can also affect seek performance).
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param dataBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setDataBlockSize(JobConf job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job, dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
-   * the underlying file system.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param fileBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setFileBlockSize(JobConf job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job, fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
-   * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
-   * can affect the performance of queries.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param indexBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setIndexBlockSize(JobConf job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job, indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system
-   * default.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param replication the number of replicas for produced files
-   * @since 1.5.0
-   */
-  public static void setReplication(JobConf job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job, replication);
-  }
-
-  /**
-   * Specify a sampler to be used when writing out data. This will result in the output file having
-   * sample data.
-   *
-   * @param job The Hadoop job instance to be configured
-   * @param samplerConfig The configuration for creating sample data in the output file.
-   * @since 1.8.0
-   */
-
-  public static void setSampler(JobConf job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
-  }
-
-  @Override
-  public RecordWriter<Key,Value> getRecordWriter(FileSystem ignored, JobConf job, String name,
-      Progressable progress) throws IOException {
-    // get the path of the temporary output file
-    final Configuration conf = job;
-    final AccumuloConfiguration acuConf =
-        FileOutputConfigurator.getAccumuloConfiguration(CLASS, job);
-
-    final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
-    final Path file =
-        new Path(getWorkOutputPath(job), getUniqueName(job, "part") + "." + extension);
-    final int visCacheSize = FileOutputConfigurator.getVisibilityCacheSize(conf);
-
-    return new RecordWriter<>() {
-      RFileWriter out = null;
-
-      @Override
-      public void close(Reporter reporter) throws IOException {
-        if (out != null) {
-          out.close();
-        }
-      }
-
-      @Override
-      public void write(Key key, Value value) throws IOException {
-        if (out == null) {
-          out = RFile.newWriter().to(file.toString()).withFileSystem(file.getFileSystem(conf))
-              .withTableProperties(acuConf).withVisibilityCacheSize(visCacheSize).build();
-          out.startDefaultLocalityGroup();
-        }
-        out.append(key, value);
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
deleted file mode 100644
index 4e59225..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides keys and values of type {@link Key} and {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(JobConf, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
-  @Override
-  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
-      throws IOException {
-    log.setLevel(getLogLevel(job));
-
-    // Override the log level from the configuration as if the RangeInputSplit has one it's the more
-    // correct one to use.
-    if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit accSplit =
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-      Level level = accSplit.getLogLevel();
-      if (level != null) {
-        log.setLevel(level);
-      }
-    } else {
-      throw new IllegalArgumentException("No RecordReader for " + split.getClass());
-    }
-
-    RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<>() {
-
-      @Override
-      public boolean next(Key key, Value value) throws IOException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Entry<Key,Value> entry = scannerIterator.next();
-          key.set(currentKey = entry.getKey());
-          value.set(entry.getValue().get());
-          if (log.isTraceEnabled()) {
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          }
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      public Key createKey() {
-        return new Key();
-      }
-
-      @Override
-      public Value createValue() {
-        return new Value();
-      }
-
-    };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
deleted file mode 100644
index d0680d0..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This
- * {@link org.apache.hadoop.mapred.InputFormat} provides keys and values of type {@link Key} and
- * {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(org.apache.hadoop.mapred.JobConf, java.util.Map)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
-  /**
-   * Sets the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects on the
-   * given Hadoop configuration
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param configs the table query configs to be set on the configuration.
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(JobConf job,
-      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs) {
-    org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
-        job, configs);
-  }
-
-  @Override
-  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
-      throws IOException {
-    log.setLevel(getLogLevel(job));
-    InputFormatBase.RecordReaderBase<Key,Value> recordReader =
-        new InputFormatBase.RecordReaderBase<>() {
-
-          @Override
-          public boolean next(Key key, Value value) throws IOException {
-            if (scannerIterator.hasNext()) {
-              ++numKeysRead;
-              Map.Entry<Key,Value> entry = scannerIterator.next();
-              key.set(currentKey = entry.getKey());
-              value.set(entry.getValue().get());
-              if (log.isTraceEnabled()) {
-                log.trace(
-                    "Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-              }
-              return true;
-            }
-            return false;
-          }
-
-          @Override
-          public Key createKey() {
-            return new Key();
-          }
-
-          @Override
-          public Value createValue() {
-            return new Value();
-          }
-
-        };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
deleted file mode 100644
index 6dbecae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,577 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Progressable;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat}
- * accepts keys and values of type {@link Text} (for a table name) and {@link Mutation} from the Map
- * and Reduce functions.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(JobConf, boolean)} is set to true)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = OutputConfigurator.client(CLASS, job);
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, "
-            + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    OutputConfigurator.setConnectorInfo(CLASS, job, principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(JobConf, boolean)} is set to true)
-   * @param tokenFile the path to the password file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    OutputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobConf job) {
-    return OutputConfigurator.isConnectorInfoSet(CLASS, job);
-  }
-
-  /**
-   * Gets the principal from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobConf job) {
-    return OutputConfigurator.getPrincipal(CLASS, job);
-  }
-
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static String getTokenClass(JobConf job) {
-    return getAuthenticationToken(job).getClass().getName();
-  }
-
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static byte[] getToken(JobConf job) {
-    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   * @see #setConnectorInfo(JobConf, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    AuthenticationToken token = OutputConfigurator.getAuthenticationToken(CLASS, job);
-    return OutputConfigurator.unwrapAuthenticationToken(job, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use
-   *             {@link #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
-   *             instead.
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   *
-   * @param clientConfig client configuration for specifying connection timeouts, SSL connection
-   *        options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(JobConf job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
-    return OutputConfigurator.getInstance(CLASS, job);
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(JobConf job, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, job, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(JobConf, Level)
-   */
-  protected static Level getLogLevel(JobConf job) {
-    return OutputConfigurator.getLogLevel(CLASS, job);
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given
-   * mutation. Table names can only be alpha-numeric and underscores.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setDefaultTableName(JobConf job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job, tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the default table name
-   * @since 1.5.0
-   * @see #setDefaultTableName(JobConf, String)
-   */
-  protected static String getDefaultTableName(JobConf job) {
-    return OutputConfigurator.getDefaultTableName(CLASS, job);
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
-   * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
-   * multiple times overwrites any previous configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param bwConfig the configuration for the {@link BatchWriter}
-   * @since 1.5.0
-   */
-  public static void setBatchWriterOptions(JobConf job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job, bwConfig);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the configuration object
-   * @since 1.5.0
-   * @see #setBatchWriterOptions(JobConf, BatchWriterConfig)
-   */
-  protected static BatchWriterConfig getBatchWriterOptions(JobConf job) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, job);
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
-   * and underscores.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setCreateTables(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.5.0
-   * @see #setCreateTables(JobConf, boolean)
-   */
-  protected static Boolean canCreateTables(JobConf job) {
-    return OutputConfigurator.canCreateTables(CLASS, job);
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is
-   * produced. This is useful for testing.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setSimulationMode(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setSimulationMode(JobConf, boolean)
-   */
-  protected static Boolean getSimulationMode(JobConf job) {
-    return OutputConfigurator.getSimulationMode(CLASS, job);
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  protected static class AccumuloRecordWriter implements RecordWriter<Text,Mutation> {
-    private MultiTableBatchWriter mtbw = null;
-    private HashMap<Text,BatchWriter> bws = null;
-    private Text defaultTableName = null;
-
-    private boolean simulate = false;
-    private boolean createTables = false;
-
-    private long mutCount = 0;
-    private long valCount = 0;
-
-    private AccumuloClient client;
-
-    protected AccumuloRecordWriter(JobConf job)
-        throws AccumuloException, AccumuloSecurityException, IOException {
-      Level l = getLogLevel(job);
-      if (l != null) {
-        log.setLevel(getLogLevel(job));
-      }
-      this.simulate = getSimulationMode(job);
-      this.createTables = canCreateTables(job);
-
-      if (simulate) {
-        log.info("Simulating output only. No writes to tables will occur");
-      }
-
-      this.bws = new HashMap<>();
-
-      String tname = getDefaultTableName(job);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.client = OutputConfigurator.client(CLASS, job);
-        mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(job));
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If
-     * {@link AccumuloOutputFormat#canCreateTables(JobConf)} is set, the table will be created if it
-     * does not exist. The table name must only contain alphanumerics and underscore.
-     */
-    @Override
-    public void write(Text table, Mutation mutation) throws IOException {
-      if (table == null || table.getLength() == 0) {
-        table = this.defaultTableName;
-      }
-
-      if (!simulate && table == null) {
-        throw new IOException("No table or default table specified. Try simulation mode next time");
-      }
-
-      ++mutCount;
-      valCount += mutation.size();
-      printMutation(table, mutation);
-
-      if (simulate) {
-        return;
-      }
-
-      if (!bws.containsKey(table)) {
-        try {
-          addTable(table);
-        } catch (final Exception e) {
-          log.error("Could not add table '" + table + "'", e);
-          throw new IOException(e);
-        }
-      }
-
-      try {
-        bws.get(table).addMutation(mutation);
-      } catch (MutationsRejectedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
-      if (simulate) {
-        log.info("Simulating adding table: " + tableName);
-        return;
-      }
-
-      log.debug("Adding table: " + tableName);
-      BatchWriter bw = null;
-      String table = tableName.toString();
-
-      if (createTables && !client.tableOperations().exists(table)) {
-        try {
-          client.tableOperations().create(table);
-        } catch (AccumuloSecurityException e) {
-          log.error("Accumulo security violation creating " + table, e);
-          throw e;
-        } catch (TableExistsException e) {
-          // Shouldn't happen
-        }
-      }
-
-      try {
-        bw = mtbw.getBatchWriter(table);
-      } catch (TableNotFoundException e) {
-        log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
-        throw new AccumuloException(e);
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw e;
-      }
-
-      if (bw != null) {
-        bws.put(tableName, bw);
-      }
-    }
-
-    private int printMutation(Text table, Mutation m) {
-      if (log.isTraceEnabled()) {
-        log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
-        for (ColumnUpdate cu : m.getUpdates()) {
-          log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()),
-              hexDump(cu.getColumnQualifier())));
-          log.trace(String.format("Table %s security: %s", table,
-              new ColumnVisibility(cu.getColumnVisibility())));
-          log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
-        }
-      }
-      return m.getUpdates().size();
-    }
-
-    private String hexDump(byte[] ba) {
-      StringBuilder sb = new StringBuilder();
-      for (byte b : ba) {
-        if ((b > 0x20) && (b < 0x7e)) {
-          sb.append((char) b);
-        } else {
-          sb.append(String.format("x%02x", b));
-        }
-      }
-      return sb.toString();
-    }
-
-    @Override
-    public void close(Reporter reporter) throws IOException {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate) {
-        return;
-      }
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (!e.getSecurityErrorCodes().isEmpty()) {
-          var tables = new HashMap<String,Set<SecurityErrorCode>>();
-          e.getSecurityErrorCodes().forEach((tabletId, secSet) -> {
-            var tableId = tabletId.getTable().canonical();
-            tables.computeIfAbsent(tableId, p -> new HashSet<>()).addAll(secSet);
-          });
-          log.error("Not authorized to write to tables : " + tables);
-        }
-
-        if (!e.getConstraintViolationSummaries().isEmpty()) {
-          log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
-        }
-        throw new IOException(e);
-      } finally {
-        client.close();
-      }
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
-    if (!isConnectorInfoSet(job)) {
-      throw new IOException("Connector info has not been set.");
-    }
-    try {
-      AccumuloClient c = OutputConfigurator.client(CLASS, job);
-      String principal = getPrincipal(job);
-      AuthenticationToken token = getAuthenticationToken(job);
-      if (!c.securityOperations().authenticateUser(principal, token)) {
-        throw new IOException("Unable to authenticate user");
-      }
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public RecordWriter<Text,Mutation> getRecordWriter(FileSystem ignored, JobConf job, String name,
-      Progressable progress) throws IOException {
-    try {
-      return new AccumuloRecordWriter(job);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
deleted file mode 100644
index e8fa3e8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides row names as {@link Text} as keys, and a corresponding {@link PeekingIterator} as a
- * value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map
- * function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(JobConf, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormat
-    extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
-  @Override
-  public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> getRecordReader(InputSplit split,
-      JobConf job, Reporter reporter) throws IOException {
-    log.setLevel(getLogLevel(job));
-    RecordReaderBase<Text,PeekingIterator<Entry<Key,Value>>> recordReader =
-        new RecordReaderBase<>() {
-          RowIterator rowIterator;
-
-          @Override
-          public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-            super.initialize(inSplit, job);
-            rowIterator = new RowIterator(scannerIterator);
-          }
-
-          @Override
-          public boolean next(Text key, PeekingIterator<Entry<Key,Value>> value)
-              throws IOException {
-            if (!rowIterator.hasNext()) {
-              return false;
-            }
-            value.initialize(rowIterator.next());
-            numKeysRead = rowIterator.getKVCount();
-            key.set((currentKey = value.peek().getKey()).getRow());
-            return true;
-          }
-
-          @Override
-          public Text createKey() {
-            return new Text();
-          }
-
-          @Override
-          public PeekingIterator<Entry<Key,Value>> createValue() {
-            return new PeekingIterator<>();
-          }
-        };
-    recordReader.initialize(split, job);
-    return recordReader;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
deleted file mode 100644
index 597c38b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of
- * K,V pairs.
- * <p>
- * Subclasses must implement a {@link #getRecordReader(InputSplit, JobConf, Reporter)} to provide a
- * {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value}
- * pairs, but one must implement its {@link RecordReaderBase#next(Object, Object)} to transform them
- * to the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setInputTableName(JobConf job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job, tableName);
-  }
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the table name
-   * @since 1.5.0
-   * @see #setInputTableName(JobConf, String)
-   */
-  protected static String getInputTableName(JobConf job) {
-    return InputConfigurator.getInputTableName(CLASS, job);
-  }
-
-  /**
-   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param ranges the ranges that will be mapped over
-   * @see TableOperations#splitRangeByTablets(String, Range, int)
-   * @since 1.5.0
-   */
-  public static void setRanges(JobConf job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job, ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the ranges
-   * @throws IOException if the ranges have been encoded improperly
-   * @since 1.5.0
-   * @see #setRanges(JobConf, Collection)
-   */
-  protected static List<Range> getRanges(JobConf job) throws IOException {
-    return InputConfigurator.getRanges(CLASS, job);
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param columnFamilyColumnQualifierPairs a pair of {@link Text} objects corresponding to column
-   *        family and column qualifier. If the column qualifier is null, the entire column family
-   *        is selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.5.0
-   */
-  public static void fetchColumns(JobConf job,
-      Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return a set of columns
-   * @since 1.5.0
-   * @see #fetchColumns(JobConf, Collection)
-   */
-  protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
-    return InputConfigurator.getFetchedColumns(CLASS, job);
-  }
-
-  /**
-   * Encode an iterator on the input for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param cfg the configuration of the iterator
-   * @since 1.5.0
-   */
-  public static void addIterator(JobConf job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job, cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return a list of iterators
-   * @since 1.5.0
-   * @see #addIterator(JobConf, IteratorSetting)
-   */
-  protected static List<IteratorSetting> getIterators(JobConf job) {
-    return InputConfigurator.getIterators(CLASS, job);
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @see #setRanges(JobConf, Collection)
-   * @since 1.5.0
-   */
-  public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
-   * {@link #setBatchScan(JobConf, boolean)} is true.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(JobConf, boolean)
-   */
-  protected static boolean getAutoAdjustRanges(JobConf job) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setScanIsolation(JobConf job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setScanIsolation(JobConf, boolean)
-   */
-  protected static boolean isIsolated(JobConf job) {
-    return InputConfigurator.isIsolated(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setLocalIterators(JobConf job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setLocalIterators(JobConf, boolean)
-   */
-  protected static boolean usesLocalIterators(JobConf job) {
-    return InputConfigurator.usesLocalIterators(CLASS, job);
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setOfflineTableScan(JobConf, boolean)
-   */
-  protected static boolean isOfflineScan(JobConf job) {
-    return InputConfigurator.isOfflineScan(CLASS, job);
-  }
-
-  /**
-   * Controls the use of the {@link BatchScanner} in this job. Using this feature will group Ranges
-   * by their source tablet, producing an InputSplit per tablet rather than per Range. This batching
-   * helps to reduce overhead when querying a large number of small ranges. (ex: when doing
-   * quad-tree decomposition for spatial queries)
-   * <p>
-   * In order to achieve good locality of InputSplits this option always clips the input Ranges to
-   * tablet boundaries. This may result in one input Range contributing to several InputSplits.
-   * <p>
-   * Note: that the value of {@link #setAutoAdjustRanges(JobConf, boolean)} is ignored and is
-   * assumed to be true when BatchScan option is enabled.
-   * <p>
-   * This configuration is incompatible with:
-   * <ul>
-   * <li>{@link #setOfflineTableScan(JobConf, boolean)}</li>
-   * <li>{@link #setLocalIterators(JobConf, boolean)}</li>
-   * <li>{@link #setScanIsolation(JobConf, boolean)}</li>
-   * </ul>
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.7.0
-   */
-  public static void setBatchScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the {@link BatchScanner} feature enabled.
-   *
-   * @param job the Hadoop context for the configured job
-   * @since 1.7.0
-   * @see #setBatchScan(JobConf, boolean)
-   */
-  public static boolean isBatchScan(JobConf job) {
-    return InputConfigurator.isBatchScan(CLASS, job);
-  }
-
-  /**
-   * Causes input format to read sample data. If sample data was created using a different
-   * configuration or a tables sampler configuration changes while reading data, then the input
-   * format will throw an error.
-   *
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param samplerConfig The sampler configuration that sample must have been created with inorder
-   *        for reading sample data to succeed.
-   *
-   * @since 1.8.0
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   */
-  public static void setSamplerConfiguration(JobConf job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
-  }
-
-  protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    @Deprecated(since = "2.0.0")
-    protected List<IteratorSetting> jobIterators(JobConf job, String tableName) {
-      return getIterators(job);
-    }
-
-    /**
-     * Apply the configured iterators to the scanner.
-     *
-     * @param iterators the iterators to set
-     * @param scanner the scanner to configure
-     * @deprecated since 1.7.0; Use {@link #jobIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Apply the configured iterators from the configuration to the scanner.
-     *
-     * @param job the job configuration
-     * @param scanner the scanner to configure
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(JobConf job, Scanner scanner) {
-      setupIterators(getIterators(job), scanner);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
deleted file mode 100644
index 3c1e749..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/RangeInputSplit.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.mapred.InputSplit;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapred instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Intended to share code between mapred and mapreduce")
-public class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit
-    implements InputSplit {
-
-  public RangeInputSplit() {}
-
-  public RangeInputSplit(RangeInputSplit split) throws IOException {
-    super(split);
-  }
-
-  protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-    super(table, tableId, range, locations);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
deleted file mode 100644
index 4324713..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ /dev/null
@@ -1,783 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.OfflineScanner;
-import org.apache.accumulo.core.clientImpl.ScannerImpl;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.dataImpl.KeyExtent;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * An abstract input format to provide shared methods common to all other input format classes. At
- * the very least, any classes inheriting from this class will need to define their own
- * {@link RecordReader}.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
-
-  private static final SecureRandom random = new SecureRandom();
-
-  protected static final Class<?> CLASS = AccumuloInputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the name of the classloader context on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param context name of the classloader context
-   * @since 1.8.0
-   */
-  public static void setClassLoaderContext(Job job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job.getConfiguration(), context);
-  }
-
-  /**
-   * Returns the name of the current classloader context set on this scanner
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return name of the current context
-   * @since 1.8.0
-   */
-  public static String getClassLoaderContext(JobContext job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(Job job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = InputConfigurator.client(CLASS, job.getConfiguration());
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, "
-            + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param tokenFile the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Job job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobContext job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobContext job) {
-    return InputConfigurator.getPrincipal(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static String getTokenClass(JobContext job) {
-    return getAuthenticationToken(job).getClass().getName();
-  }
-
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static byte[] getToken(JobContext job) {
-    return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   * @see #setConnectorInfo(Job, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobContext job) {
-    AuthenticationToken token =
-        InputConfigurator.getAuthenticationToken(CLASS, job.getConfiguration());
-    return InputConfigurator.unwrapAuthenticationToken(job, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   *
-   * @param clientConfig client configuration containing connection options
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Job job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobContext job) {
-    return InputConfigurator.getInstance(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(Job job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(Job, Level)
-   */
-  protected static Level getLogLevel(JobContext job) {
-    return InputConfigurator.getLogLevel(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization.
-   * Defaults to the empty set.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param auths the user's authorizations
-   */
-  public static void setScanAuthorizations(Job job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param job the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(Job, Authorizations)
-   */
-  protected static Authorizations getScanAuthorizations(JobContext job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Fetches all {@link InputTableConfig}s that have been set on the given job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @return the {@link InputTableConfig} objects for the job
-   * @since 1.6.0
-   */
-  protected static Map<String,InputTableConfig> getInputTableConfigs(JobContext job) {
-    return InputConfigurator.getInputTableConfigs(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Fetches a {@link InputTableConfig} that has been set on the configuration for a specific table.
-   *
-   * <p>
-   * null is returned in the event that the table doesn't exist.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table name for which to grab the config object
-   * @return the {@link InputTableConfig} for the given table
-   * @since 1.6.0
-   */
-  protected static InputTableConfig getInputTableConfig(JobContext job, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, job.getConfiguration(), tableName);
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo
-   * {@link org.apache.hadoop.mapreduce.InputFormat}.
-   *
-   * @param job the Hadoop context for the configured job
-   * @throws java.io.IOException if the context is improperly configured
-   * @since 1.5.0
-   */
-  protected static void validateOptions(JobContext job) throws IOException {
-    InputConfigurator.validatePermissions(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Construct the {@link org.apache.accumulo.core.client.ClientConfiguration} given the provided
-   * context.
-   *
-   * @param job The Job
-   * @return The ClientConfiguration
-   * @since 1.7.0
-   */
-  protected static org.apache.accumulo.core.client.ClientConfiguration
-      getClientConfiguration(JobContext job) {
-    return InputConfigurator.getClientConfiguration(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * An abstract base class to be used to create {@link org.apache.hadoop.mapreduce.RecordReader}
-   * instances that convert from Accumulo {@link Key}/{@link Value} pairs to the user's K/V types.
-   *
-   * Subclasses must implement {@link #nextKeyValue()} and use it to update the following variables:
-   * <ul>
-   * <li>K {@link #currentK}</li>
-   * <li>V {@link #currentV}</li>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  protected abstract static class AbstractRecordReader<K,V> extends RecordReader<K,V> {
-    protected long numKeysRead;
-    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
-    protected ScannerBase scannerBase;
-    protected RangeInputSplit split;
-
-    /**
-     * Extracts Iterators settings from the context to be used by RecordReader.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param tableName the table name for which the scanner is configured
-     * @return List of iterator settings for given table
-     * @since 1.7.0
-     */
-    protected abstract List<IteratorSetting> contextIterators(TaskAttemptContext context,
-        String tableName);
-
-    /**
-     * Configures the iterators on a scanner for the given table name. Will attempt to use
-     * configuration from the InputSplit, on failure will try to extract them from
-     * TaskAttemptContext.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param tableName the table name for which the scanner is configured
-     * @param scanner the scanner for which to configure the iterators
-     * @param split InputSplit containing configurations
-     * @since 1.7.0
-     */
-    private void setupIterators(TaskAttemptContext context, ScannerBase scanner, String tableName,
-        RangeInputSplit split) {
-      List<IteratorSetting> iterators = null;
-
-      if (split == null) {
-        iterators = contextIterators(context, tableName);
-      } else {
-        iterators = split.getIterators();
-        if (iterators == null) {
-          iterators = contextIterators(context, tableName);
-        }
-      }
-
-      for (IteratorSetting iterator : iterators) {
-        scanner.addScanIterator(iterator);
-      }
-    }
-
-    /**
-     * Configures the iterators on a scanner for the given table name.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param scanner the scanner for which to configure the iterators
-     * @param tableName the table name for which the scanner is configured
-     * @since 1.6.0
-     * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName,
-        RangeInputSplit split) {
-      setupIterators(context, (ScannerBase) scanner, tableName, split);
-    }
-
-    @Override
-    public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-
-      split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split);
-
-      Authorizations authorizations = getScanAuthorizations(attempt);
-      String classLoaderContext = getClassLoaderContext(attempt);
-      String table = split.getTableName();
-
-      // in case the table name changed, we can still use the previous name for terms of
-      // configuration,
-      // but the scanner will use the table id resolved at job setup time
-      InputTableConfig tableConfig = getInputTableConfig(attempt, split.getTableName());
-
-      ClientContext client = InputConfigurator.client(CLASS, split, attempt.getConfiguration());
-
-      log.debug("Created client with user: " + client.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (split instanceof org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit) {
-        org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit batchSplit =
-            (org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit) split;
-
-        BatchScanner scanner;
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner = client.createBatchScanner(split.getTableName(), authorizations, scanThreads);
-          setupIterators(attempt, scanner, split.getTableName(), split);
-          if (classLoaderContext != null) {
-            scanner.setClassLoaderContext(classLoaderContext);
-          }
-        } catch (Exception e) {
-          e.printStackTrace();
-          throw new IOException(e);
-        }
-
-        scanner.setRanges(batchSplit.getRanges());
-        scannerBase = scanner;
-      } else {
-        Scanner scanner;
-
-        Boolean isOffline = split.isOffline();
-        if (isOffline == null) {
-          isOffline = tableConfig.isOfflineScan();
-        }
-
-        Boolean isIsolated = split.isIsolatedScan();
-        if (isIsolated == null) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
-
-        Boolean usesLocalIterators = split.usesLocalIterators();
-        if (usesLocalIterators == null) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
-
-        try {
-          if (isOffline) {
-            scanner = new OfflineScanner(client, TableId.of(split.getTableId()), authorizations);
-          } else {
-            // Not using public API to create scanner so that we can use table ID
-            // Table ID is used in case of renames during M/R job
-            scanner = new ScannerImpl(client, TableId.of(split.getTableId()), authorizations);
-          }
-          if (isIsolated) {
-            log.info("Creating isolated scanner");
-            @SuppressWarnings("resource")
-            var wrapped = new IsolatedScanner(scanner);
-            scanner = wrapped;
-          }
-          if (usesLocalIterators) {
-            log.info("Using local iterators");
-            scanner = new ClientSideIteratorScanner(scanner);
-          }
-
-          setupIterators(attempt, scanner, split.getTableName(), split);
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
-
-        scanner.setRange(split.getRange());
-        scannerBase = scanner;
-
-      }
-
-      Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
-      if (columns == null) {
-        columns = tableConfig.getFetchedColumns();
-      }
-
-      // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : columns) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
-        }
-      }
-
-      SamplerConfiguration samplerConfig = split.getSamplerConfiguration();
-      if (samplerConfig == null) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
-
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
-    }
-
-    @Override
-    public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      if (numKeysRead > 0 && currentKey == null) {
-        return 1.0f;
-      }
-      return split.getProgress(currentKey);
-    }
-
-    /**
-     * The Key that should be returned to the client
-     */
-    protected K currentK = null;
-
-    /**
-     * The Value that should be return to the client
-     */
-    protected V currentV = null;
-
-    /**
-     * The Key that is used to determine progress in the current InputSplit. It is not returned to
-     * the client and is only used internally
-     */
-    protected Key currentKey = null;
-
-    @Override
-    public K getCurrentKey() throws IOException, InterruptedException {
-      return currentK;
-    }
-
-    @Override
-    public V getCurrentValue() throws IOException, InterruptedException {
-      return currentV;
-    }
-  }
-
-  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext job, TableId tableId,
-      List<Range> ranges)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    return InputConfigurator.binOffline(tableId, ranges,
-        InputConfigurator.client(CLASS, job.getConfiguration()));
-  }
-
-  /**
-   * Gets the splits of the tables that have been set on the job by reading the metadata table for
-   * the specified ranges.
-   *
-   * @return the splits from the tables based on the ranges.
-   * @throws java.io.IOException if a table set on the job doesn't exist or an error occurs
-   *         initializing the tablet locator
-   */
-  @Override
-  public List<InputSplit> getSplits(JobContext job) throws IOException {
-    Level logLevel = getLogLevel(job);
-    log.setLevel(logLevel);
-    validateOptions(job);
-
-    LinkedList<InputSplit> splits = new LinkedList<>();
-    Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(job);
-
-    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-
-      String tableName = tableConfigEntry.getKey();
-      InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-      ClientContext client;
-      try {
-        client = InputConfigurator.client(CLASS, job.getConfiguration());
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IOException(e);
-      }
-
-      TableId tableId;
-      // resolve table name to id once, and use id from this point forward
-      try {
-        tableId = client.getTableId(tableName);
-      } catch (TableNotFoundException e) {
-        throw new IOException(e);
-      }
-
-      boolean batchScan = InputConfigurator.isBatchScan(CLASS, job.getConfiguration());
-      boolean supportBatchScan = !(tableConfig.isOfflineScan()
-          || tableConfig.shouldUseIsolatedScanners() || tableConfig.shouldUseLocalIterators());
-      if (batchScan && !supportBatchScan) {
-        throw new IllegalArgumentException("BatchScanner optimization not available for offline"
-            + " scan, isolated, or local iterators");
-      }
-
-      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-      if (batchScan && !autoAdjust) {
-        throw new IllegalArgumentException(
-            "AutoAdjustRanges must be enabled when using BatchScanner optimization");
-      }
-
-      List<Range> ranges =
-          autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
-      if (ranges.isEmpty()) {
-        ranges = new ArrayList<>(1);
-        ranges.add(new Range());
-      }
-
-      // get the metadata information for these ranges
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-      TabletLocator tl;
-      try {
-        if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(job, tableId, ranges);
-          while (binnedRanges == null) {
-            // Some tablets were still online, try again
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            binnedRanges = binOfflineTable(job, tableId, ranges);
-          }
-        } else {
-          tl = TabletLocator.getLocator(client, tableId);
-          // its possible that the cache could contain complete, but old information about a
-          // tables tablets... so clear it
-          tl.invalidateCache();
-
-          while (!tl.binRanges(client, ranges, binnedRanges).isEmpty()) {
-            client.requireNotDeleted(tableId);
-            client.requireNotOffline(tableId, tableName);
-            binnedRanges.clear();
-            log.warn("Unable to locate bins for specified ranges. Retrying.");
-            // sleep randomly between 100 and 200 ms
-            sleepUninterruptibly(100 + random.nextInt(100), TimeUnit.MILLISECONDS);
-            tl.invalidateCache();
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-
-      // all of this code will add either range per each locations or split ranges and add
-      // range-location split
-      // Map from Range to Array of Locations, we only use this if we're don't split
-      HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
-      if (!autoAdjust) {
-        splitsToAdd = new HashMap<>();
-      }
-
-      HashMap<String,String> hostNameCache = new HashMap<>();
-      for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
-        String ip = tserverBin.getKey().split(":", 2)[0];
-        String location = hostNameCache.get(ip);
-        if (location == null) {
-          InetAddress inetAddress = InetAddress.getByName(ip);
-          location = inetAddress.getCanonicalHostName();
-          hostNameCache.put(ip, location);
-        }
-        for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-          Range ke = extentRanges.getKey().toDataRange();
-          if (batchScan) {
-            // group ranges by tablet to be read by a BatchScanner
-            ArrayList<Range> clippedRanges = new ArrayList<>();
-            for (Range r : extentRanges.getValue()) {
-              clippedRanges.add(ke.clip(r));
-            }
-            org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit split =
-                new org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit(tableName,
-                    tableId, clippedRanges, new String[] {location});
-            org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-                logLevel);
-
-            splits.add(split);
-          } else {
-            // not grouping by tablet
-            for (Range r : extentRanges.getValue()) {
-              if (autoAdjust) {
-                // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-                    ke.clip(r), new String[] {location});
-                org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split,
-                    tableConfig, logLevel);
-                split.setOffline(tableConfig.isOfflineScan());
-                split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-                splits.add(split);
-              } else {
-                // don't divide ranges
-                ArrayList<String> locations = splitsToAdd.get(r);
-                if (locations == null) {
-                  locations = new ArrayList<>(1);
-                }
-                locations.add(location);
-                splitsToAdd.put(r, locations);
-              }
-            }
-          }
-        }
-      }
-
-      if (!autoAdjust) {
-        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-          RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonical(),
-              entry.getKey(), entry.getValue().toArray(new String[0]));
-          org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.updateSplit(split, tableConfig,
-              logLevel);
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-
-          splits.add(split);
-        }
-      }
-    }
-    return splits;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
deleted file mode 100644
index a50beab..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.accumulo.core.client.rfile.RFile;
-import org.apache.accumulo.core.client.rfile.RFileWriter;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to write output in the Accumulo data file format.<br>
- * Care should be taken to write only sorted data (sorted by {@link Key}), as this is an important
- * requirement of Accumulo data files.
- *
- * <p>
- * The output path to be created must be specified via
- * {@link AccumuloFileOutputFormat#setOutputPath(Job, Path)}. This is inherited from
- * {@link FileOutputFormat#setOutputPath(Job, Path)}. Other methods from {@link FileOutputFormat}
- * are not supported and may be ignored or cause failures. Using other Hadoop configuration options
- * that affect the behavior of the underlying files directly in the Job's configuration may work,
- * but are not directly supported at this time.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
-
-  private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require
-   * additional libraries to be available to your Job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
-   * @since 1.5.0
-   */
-  public static void setCompressionType(Job job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job.getConfiguration(), compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
-   * a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size
-   * of the indexes (which can also affect seek performance).
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param dataBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setDataBlockSize(Job job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job.getConfiguration(), dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
-   * the underlying file system.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param fileBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setFileBlockSize(Job job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job.getConfiguration(), fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
-   * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
-   * can affect the performance of queries.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param indexBlockSize the block size, in bytes
-   * @since 1.5.0
-   */
-  public static void setIndexBlockSize(Job job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job.getConfiguration(), indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system
-   * default.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param replication the number of replicas for produced files
-   * @since 1.5.0
-   */
-  public static void setReplication(Job job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job.getConfiguration(), replication);
-  }
-
-  /**
-   * Specify a sampler to be used when writing out data. This will result in the output file having
-   * sample data.
-   *
-   * @param job The Hadoop job instance to be configured
-   * @param samplerConfig The configuration for creating sample data in the output file.
-   * @since 1.8.0
-   */
-
-  public static void setSampler(Job job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  @Override
-  public RecordWriter<Key,Value> getRecordWriter(TaskAttemptContext context) throws IOException {
-    // get the path of the temporary output file
-    final Configuration conf = context.getConfiguration();
-    final AccumuloConfiguration acuConf =
-        FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
-
-    final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
-    final Path file = this.getDefaultWorkFile(context, "." + extension);
-    final int visCacheSize = FileOutputConfigurator.getVisibilityCacheSize(conf);
-
-    return new RecordWriter<>() {
-      RFileWriter out = null;
-
-      @Override
-      public void close(TaskAttemptContext context) throws IOException {
-        if (out != null) {
-          out.close();
-        }
-      }
-
-      @Override
-      public void write(Key key, Value value) throws IOException {
-        if (out == null) {
-          out = RFile.newWriter().to(file.toString()).withFileSystem(file.getFileSystem(conf))
-              .withTableProperties(acuConf).withVisibilityCacheSize(visCacheSize).build();
-          out.startDefaultLocalityGroup();
-        }
-        out.append(key, value);
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
deleted file mode 100644
index 127a94e..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.log4j.Level;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides keys and values of type {@link Key} and {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
-
-  @Override
-  public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context)
-      throws IOException, InterruptedException {
-    log.setLevel(getLogLevel(context));
-
-    // Override the log level from the configuration as if the InputSplit has one it's the more
-    // correct one to use.
-    if (split instanceof org.apache.accumulo.core.client.mapreduce.RangeInputSplit) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit accSplit =
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-      Level level = accSplit.getLogLevel();
-      if (level != null) {
-        log.setLevel(level);
-      }
-    } else {
-      throw new IllegalArgumentException("No RecordReader for " + split.getClass());
-    }
-
-    return new RecordReaderBase<>() {
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Entry<Key,Value> entry = scannerIterator.next();
-          currentK = currentKey = entry.getKey();
-          currentV = entry.getValue();
-          if (log.isTraceEnabled()) {
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          }
-          return true;
-        }
-        return false;
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
deleted file mode 100644
index b3020e5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.format.DefaultFormatter;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This
- * {@link org.apache.hadoop.mapreduce.InputFormat} provides keys and values of type {@link Key} and
- * {@link Value} to the Map function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(Job, Map)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
-
-  /**
-   * Sets the {@link InputTableConfig} objects on the given Hadoop configuration
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param configs the table query configs to be set on the configuration.
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(Job job, Map<String,InputTableConfig> configs) {
-    requireNonNull(configs);
-    org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
-        job.getConfiguration(), configs);
-  }
-
-  @Override
-  public RecordReader<Key,Value> createRecordReader(InputSplit inputSplit,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    log.setLevel(getLogLevel(context));
-    return new AbstractRecordReader<>() {
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (scannerIterator.hasNext()) {
-          ++numKeysRead;
-          Map.Entry<Key,Value> entry = scannerIterator.next();
-          currentK = currentKey = entry.getKey();
-          currentV = entry.getValue();
-          if (log.isTraceEnabled()) {
-            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
-          }
-          return true;
-        }
-        return false;
-      }
-
-      @Override
-      protected List<IteratorSetting> contextIterators(TaskAttemptContext context,
-          String tableName) {
-        return getInputTableConfig(context, tableName).getIterators();
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
deleted file mode 100644
index c39c101..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ /dev/null
@@ -1,581 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.security.SecurityErrorCode;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.client.security.tokens.DelegationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.security.token.Token;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the sink for data. This {@link OutputFormat}
- * accepts keys and values of type {@link Text} (for a table name) and {@link Mutation} from the Map
- * and Reduce functions.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, org.apache.accumulo.core.client.ClientConfiguration)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
-
-  private static final Class<?> CLASS = AccumuloOutputFormat.class;
-  protected static final Logger log = Logger.getLogger(CLASS);
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> Some tokens, when serialized, divulge sensitive information in the
-   * configuration as a means to pass the token to MapReduce tasks. This information is BASE64
-   * encoded to provide a charset safe conversion to a string, but this conversion is not intended
-   * to be secure. {@link PasswordToken} is one example that is insecure in this way; however
-   * {@link DelegationToken}s, acquired using
-   * {@link SecurityOperations#getDelegationToken(DelegationTokenConfig)}, is not subject to this
-   * concern.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(Job, boolean)} is set to true)
-   * @param token the user's password
-   * @since 1.5.0
-   */
-  public static void setConnectorInfo(Job job, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    if (token instanceof KerberosToken) {
-      log.info("Received KerberosToken, attempting to fetch DelegationToken");
-      try {
-        ClientContext client = OutputConfigurator.client(CLASS, job.getConfiguration());
-        token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-      } catch (Exception e) {
-        log.warn("Failed to automatically obtain DelegationToken, "
-            + "Mappers/Reducers will likely fail to communicate with Accumulo", e);
-      }
-    }
-    // DelegationTokens can be passed securely from user to task without serializing insecurely in
-    // the configuration
-    if (token instanceof DelegationTokenImpl) {
-      DelegationTokenImpl delegationToken = (DelegationTokenImpl) token;
-
-      // Convert it into a Hadoop Token
-      AuthenticationTokenIdentifier identifier = delegationToken.getIdentifier();
-      Token<AuthenticationTokenIdentifier> hadoopToken = new Token<>(identifier.getBytes(),
-          delegationToken.getPassword(), identifier.getKind(), delegationToken.getServiceName());
-
-      // Add the Hadoop Token to the Job so it gets serialized and passed along.
-      job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
-    }
-
-    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt
-   * to be more secure than storing it in the Configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param principal a valid Accumulo user name (user must have Table.CREATE permission if
-   *        {@link #setCreateTables(Job, boolean)} is set to true)
-   * @param tokenFile the path to the token file
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Job job, String principal, String tokenFile)
-      throws AccumuloSecurityException {
-    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
-  }
-
-  /**
-   * Determines if the connector has been configured.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobContext context) {
-    return OutputConfigurator.isConnectorInfoSet(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobContext context) {
-    return OutputConfigurator.getPrincipal(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static String getTokenClass(JobContext context) {
-    return getAuthenticationToken(context).getClass().getName();
-  }
-
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   *
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobContext)} instead.
-   */
-  @Deprecated(since = "1.6.0")
-  protected static byte[] getToken(JobContext context) {
-    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(Job, String, AuthenticationToken)
-   * @see #setConnectorInfo(Job, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobContext context) {
-    AuthenticationToken token =
-        OutputConfigurator.getAuthenticationToken(CLASS, context.getConfiguration());
-    return OutputConfigurator.unwrapAuthenticationToken(context, token);
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param instanceName the Accumulo instance name
-   * @param zooKeepers a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use
-   *             {@link #setZooKeeperInstance(Job, org.apache.accumulo.core.client.ClientConfiguration)}
-   *             instead.
-   */
-  @Deprecated(since = "1.6.0")
-  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
-        .withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   *
-   * @param clientConfig client configuration for specifying connection timeouts, SSL connection
-   *        options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Job job,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
-  }
-
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the
-   * configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   */
-  protected static org.apache.accumulo.core.client.Instance getInstance(JobContext context) {
-    return OutputConfigurator.getInstance(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param level the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(Job job, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(Job, Level)
-   */
-  protected static Level getLogLevel(JobContext context) {
-    return OutputConfigurator.getLogLevel(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the default table name to use if one emits a null in place of a table name for a given
-   * mutation. Table names can only be alpha-numeric and underscores.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setDefaultTableName(Job job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job.getConfiguration(), tableName);
-  }
-
-  /**
-   * Gets the default table name from the configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the default table name
-   * @since 1.5.0
-   * @see #setDefaultTableName(Job, String)
-   */
-  protected static String getDefaultTableName(JobContext context) {
-    return OutputConfigurator.getDefaultTableName(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new
-   * {@link BatchWriterConfig}, with sensible built-in defaults is used. Setting the configuration
-   * multiple times overwrites any previous configuration.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param bwConfig the configuration for the {@link BatchWriter}
-   * @since 1.5.0
-   */
-  public static void setBatchWriterOptions(Job job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job.getConfiguration(), bwConfig);
-  }
-
-  /**
-   * Gets the {@link BatchWriterConfig} settings.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the configuration object
-   * @since 1.5.0
-   * @see #setBatchWriterOptions(Job, BatchWriterConfig)
-   */
-  protected static BatchWriterConfig getBatchWriterOptions(JobContext context) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric
-   * and underscores.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setCreateTables(Job job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether tables are permitted to be created as needed.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is disabled, false otherwise
-   * @since 1.5.0
-   * @see #setCreateTables(Job, boolean)
-   */
-  protected static Boolean canCreateTables(JobContext context) {
-    return OutputConfigurator.canCreateTables(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the directive to use simulation mode for this job. In simulation mode, no output is
-   * produced. This is useful for testing.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setSimulationMode(Job job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether this feature is enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setSimulationMode(Job, boolean)
-   */
-  protected static Boolean getSimulationMode(JobContext context) {
-    return OutputConfigurator.getSimulationMode(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * A base class to be used to create {@link RecordWriter} instances that write to Accumulo.
-   */
-  protected static class AccumuloRecordWriter extends RecordWriter<Text,Mutation> {
-    private MultiTableBatchWriter mtbw = null;
-    private HashMap<Text,BatchWriter> bws = null;
-    private Text defaultTableName = null;
-
-    private boolean simulate = false;
-    private boolean createTables = false;
-
-    private long mutCount = 0;
-    private long valCount = 0;
-
-    private AccumuloClient client;
-
-    protected AccumuloRecordWriter(TaskAttemptContext context)
-        throws AccumuloException, AccumuloSecurityException, IOException {
-      Level l = getLogLevel(context);
-      if (l != null) {
-        log.setLevel(getLogLevel(context));
-      }
-      this.simulate = getSimulationMode(context);
-      this.createTables = canCreateTables(context);
-
-      if (simulate) {
-        log.info("Simulating output only. No writes to tables will occur");
-      }
-
-      this.bws = new HashMap<>();
-
-      String tname = getDefaultTableName(context);
-      this.defaultTableName = (tname == null) ? null : new Text(tname);
-
-      if (!simulate) {
-        this.client = OutputConfigurator.client(CLASS, context.getConfiguration());
-        mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(context));
-      }
-    }
-
-    /**
-     * Push a mutation into a table. If table is null, the defaultTable will be used. If
-     * {@link AccumuloOutputFormat#canCreateTables(JobContext)} is set, the table will be created if
-     * it does not exist. The table name must only contain alphanumerics and underscore.
-     */
-    @Override
-    public void write(Text table, Mutation mutation) throws IOException {
-      if (table == null || table.getLength() == 0) {
-        table = this.defaultTableName;
-      }
-
-      if (!simulate && table == null) {
-        throw new IOException("No table or default table specified. Try simulation mode next time");
-      }
-
-      ++mutCount;
-      valCount += mutation.size();
-      printMutation(table, mutation);
-
-      if (simulate) {
-        return;
-      }
-
-      if (!bws.containsKey(table)) {
-        try {
-          addTable(table);
-        } catch (Exception e) {
-          log.error("Could not add table '" + table + "'", e);
-          throw new IOException(e);
-        }
-      }
-
-      try {
-        bws.get(table).addMutation(mutation);
-      } catch (MutationsRejectedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    public void addTable(Text tableName) throws AccumuloException, AccumuloSecurityException {
-      if (simulate) {
-        log.info("Simulating adding table: " + tableName);
-        return;
-      }
-
-      log.debug("Adding table: " + tableName);
-      BatchWriter bw = null;
-      String table = tableName.toString();
-
-      if (createTables && !client.tableOperations().exists(table)) {
-        try {
-          client.tableOperations().create(table);
-        } catch (AccumuloSecurityException e) {
-          log.error("Accumulo security violation creating " + table, e);
-          throw e;
-        } catch (TableExistsException e) {
-          // Shouldn't happen
-        }
-      }
-
-      try {
-        bw = mtbw.getBatchWriter(table);
-      } catch (TableNotFoundException e) {
-        log.error("Accumulo table " + table + " doesn't exist and cannot be created.", e);
-        throw new AccumuloException(e);
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw e;
-      }
-
-      if (bw != null) {
-        bws.put(tableName, bw);
-      }
-    }
-
-    private int printMutation(Text table, Mutation m) {
-      if (log.isTraceEnabled()) {
-        log.trace(String.format("Table %s row key: %s", table, hexDump(m.getRow())));
-        for (ColumnUpdate cu : m.getUpdates()) {
-          log.trace(String.format("Table %s column: %s:%s", table, hexDump(cu.getColumnFamily()),
-              hexDump(cu.getColumnQualifier())));
-          log.trace(String.format("Table %s security: %s", table,
-              new ColumnVisibility(cu.getColumnVisibility())));
-          log.trace(String.format("Table %s value: %s", table, hexDump(cu.getValue())));
-        }
-      }
-      return m.getUpdates().size();
-    }
-
-    private String hexDump(byte[] ba) {
-      StringBuilder sb = new StringBuilder();
-      for (byte b : ba) {
-        if ((b > 0x20) && (b < 0x7e)) {
-          sb.append((char) b);
-        } else {
-          sb.append(String.format("x%02x", b));
-        }
-      }
-      return sb.toString();
-    }
-
-    @Override
-    public void close(TaskAttemptContext attempt) throws IOException, InterruptedException {
-      log.debug("mutations written: " + mutCount + ", values written: " + valCount);
-      if (simulate) {
-        return;
-      }
-
-      try {
-        mtbw.close();
-      } catch (MutationsRejectedException e) {
-        if (!e.getSecurityErrorCodes().isEmpty()) {
-          var tables = new HashMap<String,Set<SecurityErrorCode>>();
-          e.getSecurityErrorCodes().forEach((table, code) -> tables
-              .computeIfAbsent(table.getTable().canonical(), k -> new HashSet<>()).addAll(code));
-          log.error("Not authorized to write to tables : " + tables);
-        }
-
-        if (!e.getConstraintViolationSummaries().isEmpty()) {
-          log.error("Constraint violations : " + e.getConstraintViolationSummaries().size());
-        }
-        throw new IOException(e);
-      } finally {
-        client.close();
-      }
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(JobContext job) throws IOException {
-    if (!isConnectorInfoSet(job)) {
-      throw new IOException("Connector info has not been set.");
-    }
-    String principal = getPrincipal(job);
-    AuthenticationToken token = getAuthenticationToken(job);
-    try {
-      AccumuloClient c = OutputConfigurator.client(CLASS, job.getConfiguration());
-      if (!c.securityOperations().authenticateUser(principal, token)) {
-        throw new IOException("Unable to authenticate user");
-      }
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public OutputCommitter getOutputCommitter(TaskAttemptContext context) {
-    return new NullOutputFormat<Text,Mutation>().getOutputCommitter(context);
-  }
-
-  @Override
-  public RecordWriter<Text,Mutation> getRecordWriter(TaskAttemptContext attempt)
-      throws IOException {
-    try {
-      return new AccumuloRecordWriter(attempt);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
deleted file mode 100644
index 0a97744..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.RowIterator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This class allows MapReduce jobs to use Accumulo as the source of data. This {@link InputFormat}
- * provides row names as {@link Text} as keys, and a corresponding {@link PeekingIterator} as a
- * value, which in turn makes the {@link Key}/{@link Value} pairs for that row available to the Map
- * function.
- *
- * The user must specify the following via static configurator methods:
- *
- * <ul>
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
- * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloRowInputFormat#setInputTableName(Job, String)}
- * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(Job, Authorizations)}
- * </ul>
- *
- * Other static methods are optional.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class AccumuloRowInputFormat
-    extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
-  @Override
-  public RecordReader<Text,PeekingIterator<Entry<Key,Value>>> createRecordReader(InputSplit split,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    log.setLevel(getLogLevel(context));
-    return new RecordReaderBase<>() {
-      RowIterator rowIterator;
-
-      @Override
-      public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-        super.initialize(inSplit, attempt);
-        rowIterator = new RowIterator(scannerIterator);
-        currentK = new Text();
-        currentV = null;
-      }
-
-      @Override
-      public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (!rowIterator.hasNext()) {
-          return false;
-        }
-        currentV = new PeekingIterator<>(rowIterator.next());
-        numKeysRead = rowIterator.getKVCount();
-        currentKey = currentV.peek().getKey();
-        currentK = new Text(currentKey.getRow());
-        return true;
-      }
-    };
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
deleted file mode 100644
index f4e4023..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of
- * K,V pairs.
- * <p>
- * Subclasses must implement a {@link #createRecordReader(InputSplit, TaskAttemptContext)} to
- * provide a {@link RecordReader} for K,V.
- * <p>
- * A static base class, RecordReaderBase, is provided to retrieve Accumulo {@link Key}/{@link Value}
- * pairs, but one must implement its {@link RecordReaderBase#nextKeyValue()} to transform them to
- * the desired generic types K,V.
- * <p>
- * See {@link AccumuloInputFormat} for an example implementation.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the table name
-   * @since 1.5.0
-   * @see #setInputTableName(Job, String)
-   */
-  protected static String getInputTableName(JobContext context) {
-    return InputConfigurator.getInputTableName(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param tableName the table to use when the tablename is null in the write call
-   * @since 1.5.0
-   */
-  public static void setInputTableName(Job job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
-  }
-
-  /**
-   * Sets the input ranges to scan for the single input table associated with this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param ranges the ranges that will be mapped over
-   * @see TableOperations#splitRangeByTablets(String, Range, int)
-   * @since 1.5.0
-   */
-  public static void setRanges(Job job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return the ranges
-   * @since 1.5.0
-   * @see #setRanges(Job, Collection)
-   */
-  protected static List<Range> getRanges(JobContext context) throws IOException {
-    return InputConfigurator.getRanges(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job for the default input table.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param columnFamilyColumnQualifierPairs a pair of {@link Text} objects corresponding to column
-   *        family and column qualifier. If the column qualifier is null, the entire column family
-   *        is selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.5.0
-   */
-  public static void fetchColumns(Job job,
-      Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return a set of columns
-   * @since 1.5.0
-   * @see #fetchColumns(Job, Collection)
-   */
-  protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
-    return InputConfigurator.getFetchedColumns(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Encode an iterator on the single input table for this job.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param cfg the configuration of the iterator
-   * @since 1.5.0
-   */
-  public static void addIterator(Job job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this
-   * configuration.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return a list of iterators
-   * @since 1.5.0
-   * @see #addIterator(Job, IteratorSetting)
-   */
-  protected static List<IteratorSetting> getIterators(JobContext context) {
-    return InputConfigurator.getIterators(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @see #setRanges(Job, Collection)
-   * @since 1.5.0
-   */
-  public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled. Must be enabled when
-   * {@link #setBatchScan(Job, boolean)} is true.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(Job, boolean)
-   */
-  protected static boolean getAutoAdjustRanges(JobContext context) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setScanIsolation(Job job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setScanIsolation(Job, boolean)
-   */
-  protected static boolean isIsolated(JobContext context) {
-    return InputConfigurator.isIsolated(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setLocalIterators(Job job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setLocalIterators(Job, boolean)
-   */
-  protected static boolean usesLocalIterators(JobContext context) {
-    return InputConfigurator.usesLocalIterators(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.5.0
-   */
-  public static void setOfflineTableScan(Job job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.5.0
-   * @see #setOfflineTableScan(Job, boolean)
-   */
-  protected static boolean isOfflineScan(JobContext context) {
-    return InputConfigurator.isOfflineScan(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Controls the use of the {@link BatchScanner} in this job. Using this feature will group Ranges
-   * by their source tablet, producing an InputSplit per tablet rather than per Range. This batching
-   * helps to reduce overhead when querying a large number of small ranges. (ex: when doing
-   * quad-tree decomposition for spatial queries)
-   * <p>
-   * In order to achieve good locality of InputSplits this option always clips the input Ranges to
-   * tablet boundaries. This may result in one input Range contributing to several InputSplits.
-   * <p>
-   * Note: that the value of {@link #setAutoAdjustRanges(Job, boolean)} is ignored and is assumed to
-   * be true when BatchScan option is enabled.
-   * <p>
-   * This configuration is incompatible with:
-   * <ul>
-   * <li>{@link #setOfflineTableScan(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * <li>{@link #setLocalIterators(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * <li>{@link #setScanIsolation(org.apache.hadoop.mapreduce.Job, boolean)}</li>
-   * </ul>
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param enableFeature the feature is enabled if true, disabled otherwise
-   * @since 1.7.0
-   */
-  public static void setBatchScan(Job job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job.getConfiguration(), enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the {@link BatchScanner} feature enabled.
-   *
-   * @param context the Hadoop context for the configured job
-   * @since 1.7.0
-   * @see #setBatchScan(Job, boolean)
-   */
-  public static boolean isBatchScan(JobContext context) {
-    return InputConfigurator.isBatchScan(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Causes input format to read sample data. If sample data was created using a different
-   * configuration or a tables sampler configuration changes while reading data, then the input
-   * format will throw an error.
-   *
-   *
-   * @param job the Hadoop job instance to be configured
-   * @param samplerConfig The sampler configuration that sample must have been created with inorder
-   *        for reading sample data to succeed.
-   *
-   * @since 1.8.0
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   */
-  public static void setSamplerConfiguration(Job job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
-
-    @Override
-    @Deprecated(since = "2.0.0")
-    protected List<IteratorSetting> contextIterators(TaskAttemptContext context, String tableName) {
-      return getIterators(context);
-    }
-
-    /**
-     * Apply the configured iterators from the configuration to the scanner.
-     *
-     * @param context the Hadoop context for the configured job
-     * @param scanner the scanner to configure
-     * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
-      // tableName is given as null as it will be ignored in eventual call to #contextIterators
-      setupIterators(context, scanner, null, null);
-    }
-
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     *
-     * @deprecated since 1.7.0; Use {@link #contextIterators} instead.
-     */
-    @Deprecated(since = "1.7.0")
-    protected void setupIterators(TaskAttemptContext context, Scanner scanner,
-        org.apache.accumulo.core.client.mapreduce.RangeInputSplit split) {
-      setupIterators(context, scanner, null, split);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
deleted file mode 100644
index c48df18..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Objects;
-
-import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.IsolatedScanner;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-/**
- * This class to holds a batch scan configuration for a table. It contains all the properties needed
- * to specify how rows should be returned from the table.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class InputTableConfig implements Writable {
-
-  private List<IteratorSetting> iterators;
-  private List<Range> ranges;
-  private Collection<Pair<Text,Text>> columns;
-
-  private boolean autoAdjustRanges = true;
-  private boolean useLocalIterators = false;
-  private boolean useIsolatedScanners = false;
-  private boolean offlineScan = false;
-  private SamplerConfiguration samplerConfig = null;
-
-  public InputTableConfig() {}
-
-  /**
-   * Creates a batch scan config object out of a previously serialized batch scan config object.
-   *
-   * @param input the data input of the serialized batch scan config
-   */
-  public InputTableConfig(DataInput input) throws IOException {
-    readFields(input);
-  }
-
-  /**
-   * Sets the input ranges to scan for all tables associated with this job. This will be added to
-   * any per-table ranges that have been set using
-   *
-   * @param ranges the ranges that will be mapped over
-   * @since 1.6.0
-   */
-  public InputTableConfig setRanges(List<Range> ranges) {
-    this.ranges = ranges;
-    return this;
-  }
-
-  /**
-   * Returns the ranges to be queried in the configuration
-   */
-  public List<Range> getRanges() {
-    return ranges != null ? ranges : new ArrayList<>();
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job for the default input table.
-   *
-   * @param columns a pair of {@link Text} objects corresponding to column family and column
-   *        qualifier. If the column qualifier is null, the entire column family is selected. An
-   *        empty set is the default and is equivalent to scanning the all columns.
-   * @since 1.6.0
-   */
-  public InputTableConfig fetchColumns(Collection<Pair<Text,Text>> columns) {
-    this.columns = columns;
-    return this;
-  }
-
-  /**
-   * Returns the columns to be fetched for this configuration
-   */
-  public Collection<Pair<Text,Text>> getFetchedColumns() {
-    return columns != null ? columns : new HashSet<>();
-  }
-
-  /**
-   * Set iterators on to be used in the query.
-   *
-   * @param iterators the configurations for the iterators
-   * @since 1.6.0
-   */
-  public InputTableConfig setIterators(List<IteratorSetting> iterators) {
-    this.iterators = iterators;
-    return this;
-  }
-
-  /**
-   * Returns the iterators to be set on this configuration
-   */
-  public List<IteratorSetting> getIterators() {
-    return iterators != null ? iterators : new ArrayList<>();
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping
-   * ranges, then splits them to align with tablet boundaries. Disabling this feature will cause
-   * exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param autoAdjustRanges the feature is enabled if true, disabled otherwise
-   * @see #setRanges(java.util.List)
-   * @since 1.6.0
-   */
-  public InputTableConfig setAutoAdjustRanges(boolean autoAdjustRanges) {
-    this.autoAdjustRanges = autoAdjustRanges;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled.
-   *
-   * @return false if the feature is disabled, true otherwise
-   * @since 1.6.0
-   * @see #setAutoAdjustRanges(boolean)
-   */
-  public boolean shouldAutoAdjustRanges() {
-    return autoAdjustRanges;
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature
-   * will cause the iterator stack to be constructed within the Map task, rather than within the
-   * Accumulo TServer. To use this feature, all classes needed for those iterators must be available
-   * on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param useLocalIterators the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public InputTableConfig setUseLocalIterators(boolean useLocalIterators) {
-    this.useLocalIterators = useLocalIterators;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setUseLocalIterators(boolean)
-   */
-  public boolean shouldUseLocalIterators() {
-    return useLocalIterators;
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are
-   * scanned. This will make the map reduce job directly read the table's files. If the table is not
-   * offline, then the job will fail. If the table comes online during the map reduce job, it is
-   * likely that the job will fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in
-   * HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any
-   * iterators that are configured for the table will need to be on the mapper's classpath. The
-   * accumulo.properties may need to be on the mapper's classpath if HDFS or the Accumulo directory
-   * in HDFS are non-standard.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as
-   * the input table for a map reduce job. If you plan to map reduce over the data many times, it
-   * may be better to the compact the table, clone it, take it offline, and use the clone for all
-   * map reduce jobs. The reason to do this is that compaction will reduce each tablet in the table
-   * to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may
-   * see better read performance. Second, it will support speculative execution better. When reading
-   * an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param offlineScan the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public InputTableConfig setOfflineScan(boolean offlineScan) {
-    this.offlineScan = offlineScan;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setOfflineScan(boolean)
-   */
-  public boolean isOfflineScan() {
-    return offlineScan;
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param useIsolatedScanners the feature is enabled if true, disabled otherwise
-   * @since 1.6.0
-   */
-  public InputTableConfig setUseIsolatedScanners(boolean useIsolatedScanners) {
-    this.useIsolatedScanners = useIsolatedScanners;
-    return this;
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @return true if the feature is enabled, false otherwise
-   * @since 1.6.0
-   * @see #setUseIsolatedScanners(boolean)
-   */
-  public boolean shouldUseIsolatedScanners() {
-    return useIsolatedScanners;
-  }
-
-  /**
-   * Set the sampler configuration to use when reading from the data.
-   *
-   * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
-   * @see InputFormatBase#setSamplerConfiguration(org.apache.hadoop.mapreduce.Job,
-   *      SamplerConfiguration)
-   *
-   * @since 1.8.0
-   */
-  public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
-    this.samplerConfig = samplerConfiguration;
-  }
-
-  /**
-   *
-   * @since 1.8.0
-   */
-  public SamplerConfiguration getSamplerConfiguration() {
-    return samplerConfig;
-  }
-
-  @Override
-  public void write(DataOutput dataOutput) throws IOException {
-    if (iterators != null) {
-      dataOutput.writeInt(iterators.size());
-      for (IteratorSetting setting : iterators) {
-        setting.write(dataOutput);
-      }
-    } else {
-      dataOutput.writeInt(0);
-    }
-    if (ranges != null) {
-      dataOutput.writeInt(ranges.size());
-      for (Range range : ranges) {
-        range.write(dataOutput);
-      }
-    } else {
-      dataOutput.writeInt(0);
-    }
-    if (columns != null) {
-      dataOutput.writeInt(columns.size());
-      for (Pair<Text,Text> column : columns) {
-        if (column.getSecond() == null) {
-          dataOutput.writeInt(1);
-          column.getFirst().write(dataOutput);
-        } else {
-          dataOutput.writeInt(2);
-          column.getFirst().write(dataOutput);
-          column.getSecond().write(dataOutput);
-        }
-      }
-    } else {
-      dataOutput.writeInt(0);
-    }
-    dataOutput.writeBoolean(autoAdjustRanges);
-    dataOutput.writeBoolean(useLocalIterators);
-    dataOutput.writeBoolean(useIsolatedScanners);
-    dataOutput.writeBoolean(offlineScan);
-    if (samplerConfig == null) {
-      dataOutput.writeBoolean(false);
-    } else {
-      dataOutput.writeBoolean(true);
-      new SamplerConfigurationImpl(samplerConfig).write(dataOutput);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    // load iterators
-    long iterSize = dataInput.readInt();
-    if (iterSize > 0) {
-      iterators = new ArrayList<>();
-    }
-    for (int i = 0; i < iterSize; i++) {
-      iterators.add(new IteratorSetting(dataInput));
-    }
-    // load ranges
-    long rangeSize = dataInput.readInt();
-    if (rangeSize > 0) {
-      ranges = new ArrayList<>();
-    }
-    for (int i = 0; i < rangeSize; i++) {
-      Range range = new Range();
-      range.readFields(dataInput);
-      ranges.add(range);
-    }
-    // load columns
-    long columnSize = dataInput.readInt();
-    if (columnSize > 0) {
-      columns = new HashSet<>();
-    }
-    for (int i = 0; i < columnSize; i++) {
-      long numPairs = dataInput.readInt();
-      Text colFam = new Text();
-      colFam.readFields(dataInput);
-      if (numPairs == 1) {
-        columns.add(new Pair<>(colFam, null));
-      } else if (numPairs == 2) {
-        Text colQual = new Text();
-        colQual.readFields(dataInput);
-        columns.add(new Pair<>(colFam, colQual));
-      }
-    }
-    autoAdjustRanges = dataInput.readBoolean();
-    useLocalIterators = dataInput.readBoolean();
-    useIsolatedScanners = dataInput.readBoolean();
-    offlineScan = dataInput.readBoolean();
-
-    if (dataInput.readBoolean()) {
-      samplerConfig = new SamplerConfigurationImpl(dataInput).toSamplerConfiguration();
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    InputTableConfig that = (InputTableConfig) o;
-    return autoAdjustRanges == that.autoAdjustRanges && useLocalIterators == that.useLocalIterators
-        && useIsolatedScanners == that.useIsolatedScanners && offlineScan == that.offlineScan
-        && Objects.equals(iterators, that.iterators) && Objects.equals(ranges, that.ranges)
-        && Objects.equals(columns, that.columns)
-        && Objects.equals(samplerConfig, that.samplerConfig);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(iterators, ranges, columns, autoAdjustRanges, useLocalIterators,
-        useIsolatedScanners, offlineScan, samplerConfig);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
deleted file mode 100644
index dcdf837..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ /dev/null
@@ -1,524 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.log4j.Level;
-
-/**
- * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class RangeInputSplit extends InputSplit implements Writable {
-  private Range range;
-  private String[] locations;
-  private String tableId, tableName, instanceName, zooKeepers, principal;
-  private org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource tokenSource;
-  private String tokenFile;
-  private AuthenticationToken token;
-  private Boolean offline, isolatedScan, localIterators;
-  private Authorizations auths;
-  private Set<Pair<Text,Text>> fetchedColumns;
-  private List<IteratorSetting> iterators;
-  private SamplerConfiguration samplerConfig;
-  private Level level;
-
-  public RangeInputSplit() {
-    range = new Range();
-    locations = new String[0];
-    tableName = "";
-    tableId = "";
-  }
-
-  public RangeInputSplit(RangeInputSplit split) throws IOException {
-    this.range = split.getRange();
-    this.setLocations(split.getLocations());
-    this.setTableName(split.getTableName());
-    this.setTableId(split.getTableId());
-  }
-
-  protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-    this.range = range;
-    setLocations(locations);
-    this.tableName = table;
-    this.tableId = tableId;
-  }
-
-  public Range getRange() {
-    return range;
-  }
-
-  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-    return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getProgress(start, end,
-        position);
-  }
-
-  public float getProgress(Key currentKey) {
-    if (currentKey == null) {
-      return 0f;
-    }
-    if (range.contains(currentKey)) {
-      if (range.getStartKey() != null && range.getEndKey() != null) {
-        if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-          // just look at the row progress
-          return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(),
-              currentKey.getRowData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
-          // just look at the column family progress
-          return getProgress(range.getStartKey().getColumnFamilyData(),
-              range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-        } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)
-            != 0) {
-          // just look at the column qualifier progress
-          return getProgress(range.getStartKey().getColumnQualifierData(),
-              range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-        }
-      }
-    }
-    // if we can't figure it out, then claim no progress
-    return 0f;
-  }
-
-  /**
-   * This implementation of length is only an estimate, it does not provide exact values. Do not
-   * have your code rely on this return value.
-   */
-  @Override
-  public long getLength() throws IOException {
-    return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getRangeLength(range);
-  }
-
-  @Override
-  public String[] getLocations() throws IOException {
-    return Arrays.copyOf(locations, locations.length);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    range.readFields(in);
-    tableName = in.readUTF();
-    tableId = in.readUTF();
-    int numLocs = in.readInt();
-    locations = new String[numLocs];
-    for (int i = 0; i < numLocs; ++i) {
-      locations[i] = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      isolatedScan = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
-      offline = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
-      localIterators = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
-      int numColumns = in.readInt();
-      List<String> columns = new ArrayList<>(numColumns);
-      for (int i = 0; i < numColumns; i++) {
-        columns.add(in.readUTF());
-      }
-
-      fetchedColumns = org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator
-          .deserializeFetchedColumns(columns);
-    }
-
-    if (in.readBoolean()) {
-      String strAuths = in.readUTF();
-      auths = new Authorizations(strAuths.getBytes(UTF_8));
-    }
-
-    if (in.readBoolean()) {
-      principal = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      int ordinal = in.readInt();
-      this.tokenSource =
-          org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource
-              .values()[ordinal];
-
-      switch (this.tokenSource) {
-        case INLINE:
-          String tokenClass = in.readUTF();
-          byte[] base64TokenBytes = in.readUTF().getBytes(UTF_8);
-          byte[] tokenBytes = Base64.getDecoder().decode(base64TokenBytes);
-
-          this.token = AuthenticationTokenSerializer.deserialize(tokenClass, tokenBytes);
-          break;
-
-        case FILE:
-          this.tokenFile = in.readUTF();
-
-          break;
-        default:
-          throw new IOException("Cannot parse unknown TokenSource ordinal");
-      }
-    }
-
-    if (in.readBoolean()) {
-      instanceName = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      zooKeepers = in.readUTF();
-    }
-
-    if (in.readBoolean()) {
-      int numIterators = in.readInt();
-      iterators = new ArrayList<>(numIterators);
-      for (int i = 0; i < numIterators; i++) {
-        iterators.add(new IteratorSetting(in));
-      }
-    }
-
-    if (in.readBoolean()) {
-      level = Level.toLevel(in.readInt());
-    }
-
-    if (in.readBoolean()) {
-      samplerConfig = new SamplerConfigurationImpl(in).toSamplerConfiguration();
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    range.write(out);
-    out.writeUTF(tableName);
-    out.writeUTF(tableId);
-    out.writeInt(locations.length);
-    for (String location : locations) {
-      out.writeUTF(location);
-    }
-
-    out.writeBoolean(isolatedScan != null);
-    if (isolatedScan != null) {
-      out.writeBoolean(isolatedScan);
-    }
-
-    out.writeBoolean(offline != null);
-    if (offline != null) {
-      out.writeBoolean(offline);
-    }
-
-    out.writeBoolean(localIterators != null);
-    if (localIterators != null) {
-      out.writeBoolean(localIterators);
-    }
-
-    out.writeBoolean(fetchedColumns != null);
-    if (fetchedColumns != null) {
-      String[] cols = org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator
-          .serializeColumns(fetchedColumns);
-      out.writeInt(cols.length);
-      for (String col : cols) {
-        out.writeUTF(col);
-      }
-    }
-
-    out.writeBoolean(auths != null);
-    if (auths != null) {
-      out.writeUTF(auths.serialize());
-    }
-
-    out.writeBoolean(principal != null);
-    if (principal != null) {
-      out.writeUTF(principal);
-    }
-
-    out.writeBoolean(tokenSource != null);
-    if (tokenSource != null) {
-      out.writeInt(tokenSource.ordinal());
-
-      if (token != null && tokenFile != null) {
-        throw new IOException(
-            "Cannot use both inline AuthenticationToken and file-based AuthenticationToken");
-      } else if (token != null) {
-        out.writeUTF(token.getClass().getName());
-        out.writeUTF(
-            Base64.getEncoder().encodeToString(AuthenticationTokenSerializer.serialize(token)));
-      } else {
-        out.writeUTF(tokenFile);
-      }
-    }
-
-    out.writeBoolean(instanceName != null);
-    if (instanceName != null) {
-      out.writeUTF(instanceName);
-    }
-
-    out.writeBoolean(zooKeepers != null);
-    if (zooKeepers != null) {
-      out.writeUTF(zooKeepers);
-    }
-
-    out.writeBoolean(iterators != null);
-    if (iterators != null) {
-      out.writeInt(iterators.size());
-      for (IteratorSetting iterator : iterators) {
-        iterator.write(out);
-      }
-    }
-
-    out.writeBoolean(level != null);
-    if (level != null) {
-      out.writeInt(level.toInt());
-    }
-
-    out.writeBoolean(samplerConfig != null);
-    if (samplerConfig != null) {
-      new SamplerConfigurationImpl(samplerConfig).write(out);
-    }
-  }
-
-  /**
-   * Use {@link #getTableName}
-   *
-   * @deprecated since 1.6.1, use getTableName() instead.
-   */
-  @Deprecated(since = "1.6.1")
-  public String getTable() {
-    return getTableName();
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * Use {@link #setTableName}
-   *
-   * @deprecated since 1.6.1, use setTableName() instead.
-   */
-  @Deprecated(since = "1.6.1")
-  public void setTable(String table) {
-    setTableName(table);
-  }
-
-  public void setTableName(String table) {
-    this.tableName = table;
-  }
-
-  public void setTableId(String tableId) {
-    this.tableId = tableId;
-  }
-
-  public String getTableId() {
-    return tableId;
-  }
-
-  /**
-   * @see #getInstance(org.apache.accumulo.core.client.ClientConfiguration)
-   * @deprecated since 1.7.0, use getInstance(ClientConfiguration) instead.
-   */
-  @Deprecated(since = "1.7.0")
-  public org.apache.accumulo.core.client.Instance getInstance() {
-    return getInstance(org.apache.accumulo.core.client.ClientConfiguration.loadDefault());
-  }
-
-  public org.apache.accumulo.core.client.Instance
-      getInstance(org.apache.accumulo.core.client.ClientConfiguration base) {
-    if (null == instanceName) {
-      return null;
-    }
-
-    if (null == zooKeepers) {
-      return null;
-    }
-
-    return new org.apache.accumulo.core.client.ZooKeeperInstance(
-        base.withInstance(getInstanceName()).withZkHosts(getZooKeepers()));
-  }
-
-  public String getInstanceName() {
-    return instanceName;
-  }
-
-  public void setInstanceName(String instanceName) {
-    this.instanceName = instanceName;
-  }
-
-  public String getZooKeepers() {
-    return zooKeepers;
-  }
-
-  public void setZooKeepers(String zooKeepers) {
-    this.zooKeepers = zooKeepers;
-  }
-
-  public String getPrincipal() {
-    return principal;
-  }
-
-  public void setPrincipal(String principal) {
-    this.principal = principal;
-  }
-
-  public AuthenticationToken getToken() {
-    return token;
-  }
-
-  public void setToken(AuthenticationToken token) {
-    this.tokenSource =
-        org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource.INLINE;
-    this.token = token;
-  }
-
-  public void setToken(String tokenFile) {
-    this.tokenSource =
-        org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase.TokenSource.FILE;
-    this.tokenFile = tokenFile;
-  }
-
-  public Boolean isOffline() {
-    return offline;
-  }
-
-  public void setOffline(Boolean offline) {
-    this.offline = offline;
-  }
-
-  public void setLocations(String[] locations) {
-    this.locations = Arrays.copyOf(locations, locations.length);
-  }
-
-  public Boolean isIsolatedScan() {
-    return isolatedScan;
-  }
-
-  public void setIsolatedScan(Boolean isolatedScan) {
-    this.isolatedScan = isolatedScan;
-  }
-
-  public Authorizations getAuths() {
-    return auths;
-  }
-
-  public void setAuths(Authorizations auths) {
-    this.auths = auths;
-  }
-
-  public void setRange(Range range) {
-    this.range = range;
-  }
-
-  public Boolean usesLocalIterators() {
-    return localIterators;
-  }
-
-  public void setUsesLocalIterators(Boolean localIterators) {
-    this.localIterators = localIterators;
-  }
-
-  public Set<Pair<Text,Text>> getFetchedColumns() {
-    return fetchedColumns;
-  }
-
-  public void setFetchedColumns(Collection<Pair<Text,Text>> fetchedColumns) {
-    this.fetchedColumns = new HashSet<>();
-    for (Pair<Text,Text> columns : fetchedColumns) {
-      this.fetchedColumns.add(columns);
-    }
-  }
-
-  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
-    this.fetchedColumns = fetchedColumns;
-  }
-
-  public List<IteratorSetting> getIterators() {
-    return iterators;
-  }
-
-  public void setIterators(List<IteratorSetting> iterators) {
-    this.iterators = iterators;
-  }
-
-  public Level getLogLevel() {
-    return level;
-  }
-
-  public void setLogLevel(Level level) {
-    this.level = level;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(256);
-    sb.append("Range: ").append(range);
-    sb.append(" Locations: ").append(Arrays.asList(locations));
-    sb.append(" Table: ").append(tableName);
-    sb.append(" TableID: ").append(tableId);
-    sb.append(" InstanceName: ").append(instanceName);
-    sb.append(" zooKeepers: ").append(zooKeepers);
-    sb.append(" principal: ").append(principal);
-    sb.append(" tokenSource: ").append(tokenSource);
-    sb.append(" authenticationToken: ").append(token);
-    sb.append(" authenticationTokenFile: ").append(tokenFile);
-    sb.append(" Authorizations: ").append(auths);
-    sb.append(" offlineScan: ").append(offline);
-    sb.append(" isolatedScan: ").append(isolatedScan);
-    sb.append(" localIterators: ").append(localIterators);
-    sb.append(" fetchColumns: ").append(fetchedColumns);
-    sb.append(" iterators: ").append(iterators);
-    sb.append(" logLevel: ").append(level);
-    sb.append(" samplerConfig: ").append(samplerConfig);
-    return sb.toString();
-  }
-
-  public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
-    this.samplerConfig = samplerConfiguration;
-  }
-
-  public SamplerConfiguration getSamplerConfiguration() {
-    return samplerConfig;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
deleted file mode 100644
index fea37ae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/KeyRangePartitioner.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce.lib.partition;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * Hadoop partitioner that uses ranges based on row keys, and optionally sub-bins based on hashing.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce.partition instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class KeyRangePartitioner extends Partitioner<Key,Writable> implements Configurable {
-  private RangePartitioner rp = new RangePartitioner();
-
-  @Override
-  public int getPartition(Key key, Writable value, int numPartitions) {
-    return rp.getPartition(key.getRow(), value, numPartitions);
-  }
-
-  @Override
-  public Configuration getConf() {
-    return rp.getConf();
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    rp.setConf(conf);
-  }
-
-  /**
-   * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split
-   * points that represent ranges for partitioning
-   */
-  public static void setSplitFile(Job job, String file) {
-    RangePartitioner.setSplitFile(job, file);
-  }
-
-  /**
-   * Sets the number of random sub-bins per range
-   */
-  public static void setNumSubBins(Job job, int num) {
-    RangePartitioner.setNumSubBins(job, num);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
deleted file mode 100644
index 91ccf3c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.mapreduce.lib.partition;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.Base64;
-import java.util.Scanner;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * Hadoop partitioner that uses ranges, and optionally sub-bins based on hashing.
- *
- * @deprecated since 2.0.0; Use org.apache.accumulo.hadoop.mapreduce.partition instead from the
- *             accumulo-hadoop-mapreduce.jar
- */
-@Deprecated(since = "2.0.0")
-public class RangePartitioner extends Partitioner<Text,Writable> implements Configurable {
-  private static final String PREFIX = RangePartitioner.class.getName();
-  private static final String CUTFILE_KEY = PREFIX + ".cutFile";
-  private static final String NUM_SUBBINS = PREFIX + ".subBins";
-
-  private Configuration conf;
-
-  @Override
-  public int getPartition(Text key, Writable value, int numPartitions) {
-    try {
-      return findPartition(key, getCutPoints(), getNumSubBins());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  int findPartition(Text key, Text[] array, int numSubBins) {
-    // find the bin for the range, and guarantee it is positive
-    int index = Arrays.binarySearch(array, key);
-    index = index < 0 ? (index + 1) * -1 : index;
-
-    // both conditions work with numSubBins == 1, but this check is to avoid
-    // hashing, when we don't need to, for speed
-    if (numSubBins < 2) {
-      return index;
-    }
-    return (key.toString().hashCode() & Integer.MAX_VALUE) % numSubBins + index * numSubBins;
-  }
-
-  private int _numSubBins = 0;
-
-  private synchronized int getNumSubBins() {
-    if (_numSubBins < 1) {
-      // get number of sub-bins and guarantee it is positive
-      _numSubBins = Math.max(1, getConf().getInt(NUM_SUBBINS, 1));
-    }
-    return _numSubBins;
-  }
-
-  private Text[] cutPointArray = null;
-
-  private synchronized Text[] getCutPoints() throws IOException {
-    if (cutPointArray == null) {
-      String cutFileName = conf.get(CUTFILE_KEY);
-      TreeSet<Text> cutPoints = new TreeSet<>();
-      try (
-          InputStream inputStream =
-              DistributedCacheHelper.openCachedFile(cutFileName, CUTFILE_KEY, conf);
-          Scanner in = new Scanner(inputStream, UTF_8)) {
-        while (in.hasNextLine()) {
-          cutPoints.add(new Text(Base64.getDecoder().decode(in.nextLine())));
-        }
-      }
-
-      cutPointArray = cutPoints.toArray(new Text[cutPoints.size()]);
-
-      if (cutPointArray == null) {
-        throw new IOException("Cutpoint array not properly created from file" + cutFileName);
-      }
-    }
-    return cutPointArray;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  /**
-   * Sets the hdfs file name to use, containing a newline separated list of Base64 encoded split
-   * points that represent ranges for partitioning
-   */
-  public static void setSplitFile(Job job, String file) {
-    DistributedCacheHelper.addCacheFile(job, file, CUTFILE_KEY);
-    job.getConfiguration().set(CUTFILE_KEY, file);
-  }
-
-  /**
-   * Sets the number of random sub-bins per range
-   */
-  public static void setNumSubBins(Job job, int num) {
-    job.getConfiguration().setInt(NUM_SUBBINS, num);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
deleted file mode 100644
index 34b3052..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.replication;
-
-/**
- * The peer already exists
- */
-@Deprecated(since = "2.1.0")
-public class PeerExistsException extends Exception {
-  private static final long serialVersionUID = 1L;
-
-  public PeerExistsException(String peer) {
-    this(peer, (String) null);
-  }
-
-  public PeerExistsException(String peer, String message) {
-    super("Peer '" + peer + "' already exists"
-        + (message == null || message.isEmpty() ? "" : message));
-  }
-
-  public PeerExistsException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
deleted file mode 100644
index 4da5ec9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.client.replication;
-
-/**
- * The peer already exists
- */
-@Deprecated(since = "2.1.0")
-public class PeerNotFoundException extends Exception {
-  private static final long serialVersionUID = 1L;
-
-  public PeerNotFoundException(String peer) {
-    this(peer, (String) null);
-  }
-
-  public PeerNotFoundException(String peer, String message) {
-    super("Peer '" + peer + "' not found " + (message == null || message.isEmpty() ? "" : message));
-  }
-
-  public PeerNotFoundException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public PeerNotFoundException(String peer, String message, Throwable cause) {
-    super("Peer '" + peer + "' not found " + message, cause);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
index 3b6d10a..64956dc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFile.java
@@ -23,6 +23,7 @@
 import java.util.Collection;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.Scanner;
@@ -36,7 +37,9 @@
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.RowRangeUtil;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -76,6 +79,38 @@
      * @return this
      */
     ScannerFSOptions from(String... files);
+
+    /**
+     * Specify FencedPath files to read from. When multiple are specified the {@link Scanner}
+     * constructed will present a merged view.
+     *
+     * @param files one or more FencedPaths to read.
+     * @return this
+     *
+     * @since 3.1.0
+     */
+    ScannerFSOptions from(FencedPath... files);
+
+    /**
+     * @since 3.1.0
+     */
+    class FencedPath {
+      private final Path path;
+      private final Range fence;
+
+      public FencedPath(Path path, Range fence) {
+        this.path = Objects.requireNonNull(path);
+        this.fence = RowRangeUtil.requireKeyExtentDataRange(fence);
+      }
+
+      public Path getPath() {
+        return path;
+      }
+
+      public Range getFence() {
+        return fence;
+      }
+    }
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
index 767ebc3..b62fc84 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.client.rfile;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -140,6 +141,11 @@
         public long requestCount() {
           return 0L;
         }
+
+        @Override
+        public long evictionCount() {
+          return 0L;
+        }
       };
     }
 
@@ -211,10 +217,9 @@
         blockCacheManager.start(BlockCacheConfiguration.forTabletServer(cc));
         this.indexCache = blockCacheManager.getBlockCache(CacheType.INDEX);
         this.dataCache = blockCacheManager.getBlockCache(CacheType.DATA);
-      } catch (RuntimeException e) {
-        throw e;
-      } catch (Exception e) {
-        throw new RuntimeException(e);
+      } catch (ReflectiveOperationException e) {
+        throw new IllegalArgumentException(
+            "Configuration does not contain loadable class for block cache manager factory", e);
       }
     }
     if (indexCache == null) {
@@ -351,7 +356,7 @@
         CachableBuilder cb =
             new CachableBuilder().input(inputStream, "source-" + i).length(sources[i].getLength())
                 .conf(opts.in.getConf()).cacheProvider(cacheProvider).cryptoService(cryptoService);
-        readers.add(new RFile.Reader(cb));
+        readers.add(RFile.getReader(cb, sources[i].getRange()));
       }
 
       if (getSamplerConfiguration() != null) {
@@ -389,14 +394,14 @@
           iterator = IteratorConfigUtil.loadIterators(iterator, iteratorBuilder);
         }
       } catch (IOException e) {
-        throw new RuntimeException(e);
+        throw new UncheckedIOException(e);
       }
 
       iterator.seek(getRange() == null ? EMPTY_RANGE : getRange(), families, !families.isEmpty());
       return new IteratorAdapter(iterator);
 
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -407,14 +412,10 @@
         source.getInputStream().close();
       }
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
-    try {
-      if (this.blockCacheManager != null) {
-        this.blockCacheManager.stop();
-      }
-    } catch (Exception e1) {
-      throw new RuntimeException(e1);
+    if (this.blockCacheManager != null) {
+      this.blockCacheManager.stop();
     }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
index f4ab7e9..e63c795 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScannerBuilder.java
@@ -38,32 +38,37 @@
 class RFileScannerBuilder implements RFile.InputArguments, RFile.ScannerFSOptions {
 
   static class InputArgs extends FSConfArgs {
-    private Path[] paths;
+    private FencedPath[] rFiles;
     private RFileSource[] sources;
 
     InputArgs(String... files) {
-      this.paths = new Path[files.length];
+      this.rFiles = new FencedPath[files.length];
       for (int i = 0; i < files.length; i++) {
-        this.paths[i] = new Path(files[i]);
+        this.rFiles[i] = new FencedPath(new Path(files[i]), new Range());
       }
     }
 
+    InputArgs(FencedPath... files) {
+      this.rFiles = files;
+    }
+
     InputArgs(RFileSource... sources) {
       this.sources = sources;
     }
 
     RFileSource[] getSources() throws IOException {
       if (sources == null) {
-        sources = new RFileSource[paths.length];
-        for (int i = 0; i < paths.length; i++) {
-          sources[i] = new RFileSource(getFileSystem().open(paths[i]),
-              getFileSystem().getFileStatus(paths[i]).getLen());
+        sources = new RFileSource[rFiles.length];
+        for (int i = 0; i < rFiles.length; i++) {
+          final Path path = rFiles[i].getPath();
+          sources[i] = new RFileSource(getFileSystem().open(path),
+              getFileSystem().getFileStatus(path).getLen(), rFiles[i].getFence());
         }
       } else {
         for (int i = 0; i < sources.length; i++) {
           if (!(sources[i].getInputStream() instanceof FSDataInputStream)) {
             sources[i] = new RFileSource(new FSDataInputStream(sources[i].getInputStream()),
-                sources[i].getLength());
+                sources[i].getLength(), rFiles[i].getFence());
           }
         }
       }
@@ -128,6 +133,13 @@
   }
 
   @Override
+  public ScannerFSOptions from(FencedPath... files) {
+    Objects.requireNonNull(files);
+    opts.in = new InputArgs(files);
+    return this;
+  }
+
+  @Override
   public ScannerOptions withTableProperties(Iterable<Entry<String,String>> tableConfig) {
     Objects.requireNonNull(tableConfig);
     this.opts.tableConfig = new HashMap<>();
@@ -150,4 +162,5 @@
     this.opts.bounds = range;
     return this;
   }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
index cb5d0bb..a77e2a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileSource.java
@@ -19,6 +19,9 @@
 package org.apache.accumulo.core.client.rfile;
 
 import java.io.InputStream;
+import java.util.Objects;
+
+import org.apache.accumulo.core.data.Range;
 
 /**
  * RFile metadata is stored at the end of the file. Inorder to read an RFile, its length must be
@@ -29,10 +32,19 @@
 public class RFileSource {
   private final InputStream in;
   private final long len;
+  private final Range range;
 
   public RFileSource(InputStream in, long len) {
-    this.in = in;
+    this(in, len, new Range());
+  }
+
+  /**
+   * @since 3.1.0
+   */
+  public RFileSource(InputStream in, long len, Range range) {
+    this.in = Objects.requireNonNull(in);
     this.len = len;
+    this.range = Objects.requireNonNull(range);
   }
 
   public InputStream getInputStream() {
@@ -42,4 +54,13 @@
   public long getLength() {
     return len;
   }
+
+  /**
+   * @return The range of the RFileSource
+   *
+   * @since 3.1.0
+   */
+  public Range getRange() {
+    return range;
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
index be1850c..b1d7957 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileWriterBuilder.java
@@ -38,6 +38,7 @@
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoFactoryLoader;
 import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.metadata.ValidationUtil;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.spi.crypto.CryptoEnvironment;
@@ -119,7 +120,8 @@
           visCacheSize);
     } else {
       return new RFileWriter(fileops.newWriterBuilder()
-          .forFile(out.path.toString(), out.getFileSystem(), out.getConf(), cs)
+          .forFile(UnreferencedTabletFile.of(out.getFileSystem(), out.path), out.getFileSystem(),
+              out.getConf(), cs)
           .withTableConfiguration(acuconf).withStartDisabled().build(), visCacheSize);
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java b/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
index 072f6ca..59895e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/sample/AbstractHashSampler.java
@@ -23,6 +23,7 @@
 
 import java.io.DataOutput;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Map;
 import java.util.Set;
 
@@ -85,17 +86,6 @@
   }
 
   /**
-   * Subclasses with options should override this method and return true if the option is valid for
-   * the subclass or if {@code super.isValidOption(opt)} returns true.
-   *
-   * @deprecated since 2.1.0, replaced by {@link #validateOptions(Map)}
-   */
-  @Deprecated(since = "2.1.0")
-  protected boolean isValidOption(String option) {
-    return REQUIRED_SAMPLER_OPTIONS.contains(option);
-  }
-
-  /**
    * Subclasses with options should override this method and call {@code super.init(config)}.
    */
   @SuppressFBWarnings(value = "UNSAFE_HASH_EQUALS",
@@ -142,7 +132,7 @@
     try {
       hash(new DataoutputHasher(hasher), k);
     } catch (IOException e) {
-      throw new RuntimeException(e);
+      throw new UncheckedIOException(e);
     }
     return hasher.hash().asInt() % modulus == 0;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
index 6e53023..0652a3b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/AuthenticationToken.java
@@ -23,6 +23,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
@@ -115,7 +116,7 @@
         token.write(out);
         return baos.toByteArray();
       } catch (IOException e) {
-        throw new RuntimeException("Bug found in serialization code", e);
+        throw new UncheckedIOException("Bug found in serialization code", e);
       }
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
index 50f44c7..a28cc06 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/KerberosToken.java
@@ -25,6 +25,7 @@
 import java.io.DataOutput;
 import java.io.File;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.Collections;
 import java.util.Set;
 
@@ -107,8 +108,10 @@
       clone.principal = principal;
       clone.keytab = keytab == null ? keytab : keytab.getCanonicalFile();
       return clone;
-    } catch (CloneNotSupportedException | IOException e) {
-      throw new RuntimeException(e);
+    } catch (CloneNotSupportedException e) {
+      throw new IllegalStateException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
index 8390f0f..30e570c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
@@ -101,8 +101,8 @@
       AtomicBoolean calledFirstReadInt = new AtomicBoolean(false);
       DataInput wrapped = (DataInput) Proxy.newProxyInstance(DataInput.class.getClassLoader(),
           arg0.getClass().getInterfaces(), (obj, method, args) -> {
-            // wrap the original DataInput in order to return the integer that was read
-            // and then not used, because it didn't match -2
+            // wrap the original DataInput in order to simulate replacing the integer that was
+            // previously read and then not used back into the input, after it didn't match -2
             if (!calledFirstReadInt.get() && method.getName().equals("readInt")) {
               calledFirstReadInt.set(true);
               return version;
@@ -160,7 +160,7 @@
       clone.password = Arrays.copyOf(password, password.length);
       return clone;
     } catch (CloneNotSupportedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
index 5660f66..77a9244 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/summary/summarizers/AuthorizationSummarizer.java
@@ -24,14 +24,13 @@
 import java.util.Set;
 import java.util.function.Consumer;
 
+import org.apache.accumulo.access.AccessExpression;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.summary.CountingSummarizer;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.ColumnVisibility.Node;
 
 /**
  * Counts unique authorizations in column visibility labels. Leverages super class to defend against
@@ -82,7 +81,10 @@
       if (vis.length() > 0) {
         Set<ByteSequence> auths = cache.get(vis);
         if (auths == null) {
-          auths = findAuths(vis);
+          auths = new HashSet<>();
+          for (String auth : AccessExpression.of(vis.toArray()).getAuthorizations().asSet()) {
+            auths.add(new ArrayByteSequence(auth));
+          }
           cache.put(new ArrayByteSequence(vis), auths);
         }
 
@@ -91,33 +93,5 @@
         }
       }
     }
-
-    private Set<ByteSequence> findAuths(ByteSequence vis) {
-      HashSet<ByteSequence> auths = new HashSet<>();
-      byte[] expression = vis.toArray();
-      Node root = new ColumnVisibility(expression).getParseTree();
-
-      findAuths(root, expression, auths);
-
-      return auths;
-    }
-
-    private void findAuths(Node node, byte[] expression, HashSet<ByteSequence> auths) {
-      switch (node.getType()) {
-        case AND:
-        case OR:
-          for (Node child : node.getChildren()) {
-            findAuths(child, expression, auths);
-          }
-          break;
-        case TERM:
-          auths.add(node.getTerm(expression));
-          break;
-        case EMPTY:
-          break;
-        default:
-          throw new IllegalArgumentException("Unknown node type " + node.getType());
-      }
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
index b7889c3..ca36f3d 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveCompactionImpl.java
@@ -30,7 +30,8 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
-import org.apache.accumulo.core.util.HostAndPort;
+
+import com.google.common.net.HostAndPort;
 
 /**
  * @since 1.6.0
@@ -102,6 +103,11 @@
   }
 
   @Override
+  public long getPausedCount() {
+    return tac.getTimesPaused();
+  }
+
+  @Override
   public List<IteratorSetting> getIterators() {
     ArrayList<IteratorSetting> ret = new ArrayList<>();
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
index ba9160f..d125fd2 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ActiveScanImpl.java
@@ -57,7 +57,7 @@
   private Authorizations authorizations;
 
   ActiveScanImpl(ClientContext context,
-      org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan)
+      org.apache.accumulo.core.tabletscan.thrift.ActiveScan activeScan)
       throws TableNotFoundException {
     this.scanId = activeScan.scanId;
     this.client = activeScan.client;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
index 3929c3e..2ccc440 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl;
 
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Properties;
 import java.util.function.Predicate;
@@ -38,121 +36,74 @@
 public class ClientConfConverter {
 
   private static final Logger log = LoggerFactory.getLogger(ClientConfConverter.class);
-  private static Map<String,String> confProps = new HashMap<>();
-  private static Map<String,String> propsConf = new HashMap<>();
-
-  @SuppressWarnings("deprecation")
-  private static void init() {
-    propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_HOST
-            .getKey());
-    propsConf.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_ZK_TIMEOUT
-            .getKey());
-    propsConf.put(ClientProperty.SSL_ENABLED.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_ENABLED
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_PATH.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PATH
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_TYPE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_TYPE
-            .getKey());
-    propsConf.put(ClientProperty.SSL_KEYSTORE_PASSWORD.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_KEYSTORE_PASSWORD
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_PATH.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PATH
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_TYPE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_TYPE
-            .getKey());
-    propsConf.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD
-            .getKey());
-    propsConf.put(ClientProperty.SSL_USE_JSSE.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_USE_JSSE.getKey());
-    propsConf.put(ClientProperty.SASL_ENABLED.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SASL_ENABLED
-            .getKey());
-    propsConf.put(ClientProperty.SASL_QOP.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.RPC_SASL_QOP.getKey());
-    propsConf.put(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
-        org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-            .getKey());
-
-    for (Map.Entry<String,String> entry : propsConf.entrySet()) {
-      confProps.put(entry.getValue(), entry.getKey());
-    }
-  }
+  private static final Map<String,String> accumuloConfToClientProps = new HashMap<>();
+  private static final Map<String,String> clientPropsToAccumuloConf = new HashMap<>();
 
   static {
-    init();
-  }
+    // mapping of ClientProperty equivalents in AccumuloConfiguration
+    Map<ClientProperty,Property> conversions = new HashMap<>();
+    conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS, Property.INSTANCE_ZK_HOST);
+    conversions.put(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, Property.INSTANCE_ZK_TIMEOUT);
 
-  @SuppressWarnings("deprecation")
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      toClientConf(Properties properties) {
-    org.apache.accumulo.core.client.ClientConfiguration config =
-        org.apache.accumulo.core.client.ClientConfiguration.create();
-    for (Object keyObj : properties.keySet()) {
-      String propKey = (String) keyObj;
-      String val = properties.getProperty(propKey);
-      String confKey = propsConf.get(propKey);
-      if (confKey == null) {
-        config.setProperty(propKey, val);
-      } else {
-        config.setProperty(confKey, val);
-      }
-      if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
-        config.setProperty(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-            "true");
-      }
-    }
-    return config;
-  }
+    conversions.put(ClientProperty.SASL_ENABLED, Property.INSTANCE_RPC_SASL_ENABLED);
+    conversions.put(ClientProperty.SASL_QOP, Property.RPC_SASL_QOP);
 
-  @SuppressWarnings("deprecation")
-  public static Properties
-      toProperties(org.apache.accumulo.core.client.ClientConfiguration clientConf) {
-    Properties props = new Properties();
-    Iterator<String> clientConfIter = clientConf.getKeys();
-    while (clientConfIter.hasNext()) {
-      String confKey = clientConfIter.next();
-      String val = clientConf.getString(confKey);
-      String propKey = confProps.get(confKey);
-      if (propKey == null) {
-        if (!confKey.equals(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH
-                .getKey())) {
-          props.setProperty(confKey, val);
-        }
-      } else {
-        props.setProperty(propKey, val);
-      }
+    conversions.put(ClientProperty.SSL_ENABLED, Property.INSTANCE_RPC_SSL_ENABLED);
+    conversions.put(ClientProperty.SSL_KEYSTORE_PASSWORD, Property.RPC_SSL_KEYSTORE_PASSWORD);
+    conversions.put(ClientProperty.SSL_KEYSTORE_PATH, Property.RPC_SSL_KEYSTORE_PATH);
+    conversions.put(ClientProperty.SSL_KEYSTORE_TYPE, Property.RPC_SSL_KEYSTORE_TYPE);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_PASSWORD, Property.RPC_SSL_TRUSTSTORE_PASSWORD);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_PATH, Property.RPC_SSL_TRUSTSTORE_PATH);
+    conversions.put(ClientProperty.SSL_TRUSTSTORE_TYPE, Property.RPC_SSL_TRUSTSTORE_TYPE);
+    conversions.put(ClientProperty.SSL_USE_JSSE, Property.RPC_USE_JSSE);
+
+    for (Map.Entry<ClientProperty,Property> entry : conversions.entrySet()) {
+      accumuloConfToClientProps.put(entry.getValue().getKey(), entry.getKey().getKey());
+      clientPropsToAccumuloConf.put(entry.getKey().getKey(), entry.getValue().getKey());
     }
-    return props;
   }
 
   public static Properties toProperties(AccumuloConfiguration config) {
-    return toProperties(toClientConf(config));
+    final var propsExtractedFromConfig = new Properties();
+
+    // Extract kerberos primary from the config
+    final String serverPrincipal = config.get(Property.GENERAL_KERBEROS_PRINCIPAL);
+    if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
+      var krbName = new KerberosName(serverPrincipal);
+      propsExtractedFromConfig.setProperty(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey(),
+          krbName.getServiceName());
+    }
+
+    // Extract the remaining properties from the config
+    config.stream().filter(e -> accumuloConfToClientProps.keySet().contains(e.getKey()))
+        .forEach(e -> propsExtractedFromConfig.setProperty(e.getKey(), e.getValue()));
+
+    // For all the extracted properties, convert them to their ClientProperty names
+    final var convertedProps = new Properties();
+    propsExtractedFromConfig.forEach((k, v) -> {
+      String confKey = String.valueOf(k);
+      String val = String.valueOf(v);
+      String propKey = accumuloConfToClientProps.get(confKey);
+      convertedProps.setProperty(propKey == null ? confKey : propKey, val);
+    });
+    return convertedProps;
   }
 
   public static AccumuloConfiguration toAccumuloConf(Properties properties) {
-    return toAccumuloConf(toClientConf(properties));
-  }
-
-  /**
-   * A utility method for converting client configuration to a standard configuration object for use
-   * internally.
-   *
-   * @param config the original config
-   * @return the client configuration presented in the form of an {@link AccumuloConfiguration}
-   */
-  @SuppressWarnings("deprecation")
-  public static AccumuloConfiguration
-      toAccumuloConf(final org.apache.accumulo.core.client.ClientConfiguration config) {
+    final var convertedProps = new Properties();
+    for (String propKey : properties.stringPropertyNames()) {
+      String val = properties.getProperty(propKey);
+      String confKey = clientPropsToAccumuloConf.get(propKey);
+      if (propKey.equals(ClientProperty.SASL_KERBEROS_SERVER_PRIMARY.getKey())) {
+        confKey = Property.GENERAL_KERBEROS_PRINCIPAL.getKey();
+        // Avoid providing a realm since we don't know what it is...
+        val += "/_HOST@" + SaslConnectionParams.getDefaultRealm();
+      }
+      convertedProps.setProperty(confKey == null ? propKey : confKey, val);
+      if (propKey.equals(ClientProperty.SSL_KEYSTORE_PATH.getKey())) {
+        convertedProps.setProperty(Property.INSTANCE_RPC_SSL_CLIENT_AUTH.getKey(), "true");
+      }
+    }
 
     final AccumuloConfiguration defaults = DefaultConfiguration.getInstance();
 
@@ -160,7 +111,7 @@
 
       @Override
       public boolean isPropertySet(Property prop) {
-        return config.containsKey(prop.getKey());
+        return convertedProps.containsKey(prop.getKey());
       }
 
       @Override
@@ -181,51 +132,15 @@
             }
           }
         }
-
-        if (config.containsKey(key)) {
-          return config.getString(key);
-        } else {
-          // Reconstitute the server kerberos property from the client config
-          if (property == Property.GENERAL_KERBEROS_PRINCIPAL) {
-            if (config.containsKey(
-                org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                    .getKey())) {
-              // Avoid providing a realm since we don't know what it is...
-              return config.getString(
-                  org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                      .getKey())
-                  + "/_HOST@" + SaslConnectionParams.getDefaultRealm();
-            }
-          }
-          return defaults.get(property);
-        }
+        return convertedProps.getProperty(key, defaults.get(property));
       }
 
       @Override
       public void getProperties(Map<String,String> props, Predicate<String> filter) {
         defaults.getProperties(props, filter);
-
-        Iterator<String> keyIter = config.getKeys();
-        while (keyIter.hasNext()) {
-          String key = keyIter.next();
+        for (String key : convertedProps.stringPropertyNames()) {
           if (filter.test(key)) {
-            props.put(key, config.getString(key));
-          }
-        }
-
-        // Two client props that don't exist on the server config. Client doesn't need to know about
-        // the Kerberos instance from the principle, but servers do
-        // Automatically reconstruct the server property when converting a client config.
-        if (props.containsKey(
-            org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                .getKey())) {
-          final String serverPrimary = props.remove(
-              org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY
-                  .getKey());
-          if (filter.test(Property.GENERAL_KERBEROS_PRINCIPAL.getKey())) {
-            // Use the _HOST expansion. It should be unnecessary in "client land".
-            props.put(Property.GENERAL_KERBEROS_PRINCIPAL.getKey(),
-                serverPrimary + "/_HOST@" + SaslConnectionParams.getDefaultRealm());
+            props.put(key, convertedProps.getProperty(key));
           }
         }
 
@@ -247,8 +162,8 @@
       }
 
       private org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
-        String credProviderPaths =
-            config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+        String credProviderPaths = convertedProps
+            .getProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
         if (credProviderPaths != null && !credProviderPaths.isEmpty()) {
           org.apache.hadoop.conf.Configuration hConf = new org.apache.hadoop.conf.Configuration();
           HadoopCredentialProvider.setPath(hConf, credProviderPaths);
@@ -259,40 +174,7 @@
 
         return null;
       }
+
     };
   }
-
-  @SuppressWarnings("deprecation")
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      toClientConf(AccumuloConfiguration conf) {
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        org.apache.accumulo.core.client.ClientConfiguration.create();
-
-    // Servers will only have the full principal in their configuration -- parse the
-    // primary and realm from it.
-    final String serverPrincipal = conf.get(Property.GENERAL_KERBEROS_PRINCIPAL);
-
-    final KerberosName krbName;
-    if (serverPrincipal != null && !serverPrincipal.isEmpty()) {
-      krbName = new KerberosName(serverPrincipal);
-      clientConf.setProperty(
-          org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.KERBEROS_SERVER_PRIMARY,
-          krbName.getServiceName());
-    }
-
-    HashSet<String> clientKeys = new HashSet<>();
-    for (org.apache.accumulo.core.client.ClientConfiguration.ClientProperty prop : org.apache.accumulo.core.client.ClientConfiguration.ClientProperty
-        .values()) {
-      clientKeys.add(prop.getKey());
-    }
-
-    String key;
-    for (Map.Entry<String,String> entry : conf) {
-      key = entry.getKey();
-      if (clientKeys.contains(key)) {
-        clientConf.setProperty(key, entry.getValue());
-      }
-    }
-    return clientConf;
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 75b6394..eb77916 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -22,7 +22,6 @@
 import static com.google.common.base.Suppliers.memoize;
 import static com.google.common.base.Suppliers.memoizeWithExpiration;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
@@ -37,6 +36,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -76,12 +76,14 @@
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
 import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.lock.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLockData;
+import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.Ample;
@@ -148,8 +150,6 @@
   private final TableOperationsImpl tableops;
   private final NamespaceOperations namespaceops;
   private InstanceOperations instanceops = null;
-  @SuppressWarnings("deprecation")
-  private org.apache.accumulo.core.client.admin.ReplicationOperations replicationops = null;
   private final SingletonReservation singletonReservation;
   private final ThreadPools clientThreadPools;
   private ThreadPoolExecutor cleanupThreadPool;
@@ -406,11 +406,10 @@
       try {
         final var zLockPath = ServiceLock.path(root + "/" + addr);
         ZcStat stat = new ZcStat();
-        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
-        if (lockData != null) {
-          String[] fields = new String(lockData, UTF_8).split(",", 2);
-          UUID uuid = UUID.fromString(fields[0]);
-          String group = fields[1];
+        Optional<ServiceLockData> sld = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (sld.isPresent()) {
+          UUID uuid = sld.orElseThrow().getServerUUID(ThriftService.TABLET_SCAN);
+          String group = sld.orElseThrow().getGroup(ThriftService.TABLET_SCAN);
           liveScanServers.put(addr, new Pair<>(uuid, group));
         }
       } catch (IllegalArgumentException e) {
@@ -510,35 +509,34 @@
    */
   public List<String> getManagerLocations() {
     ensureOpen();
-    return getManagerLocations(zooCache, getInstanceID().canonical());
-  }
-
-  // available only for sharing code with old ZooKeeperInstance
-  public static List<String> getManagerLocations(ZooCache zooCache, String instanceId) {
     var zLockManagerPath =
-        ServiceLock.path(Constants.ZROOT + "/" + instanceId + Constants.ZMANAGER_LOCK);
+        ServiceLock.path(Constants.ZROOT + "/" + getInstanceID() + Constants.ZMANAGER_LOCK);
 
     OpTimer timer = null;
 
     if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up manager location in zookeeper.", Thread.currentThread().getId());
+      log.trace("tid={} Looking up manager location in zookeeper at {}.",
+          Thread.currentThread().getId(), zLockManagerPath);
       timer = new OpTimer().start();
     }
 
-    byte[] loc = zooCache.getLockData(zLockManagerPath);
+    Optional<ServiceLockData> sld = zooCache.getLockData(zLockManagerPath);
+    String location = null;
+    if (sld.isPresent()) {
+      location = sld.orElseThrow().getAddressString(ThriftService.MANAGER);
+    }
 
     if (timer != null) {
       timer.stop();
       log.trace("tid={} Found manager at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(SECONDS)));
+          (location == null ? "null" : location), String.format("%.3f secs", timer.scale(SECONDS)));
     }
 
-    if (loc == null) {
+    if (location == null) {
       return Collections.emptyList();
     }
 
-    return Collections.singletonList(new String(loc, UTF_8));
+    return Collections.singletonList(location);
   }
 
   /**
@@ -549,37 +547,27 @@
   public InstanceId getInstanceID() {
     ensureOpen();
     if (instanceId == null) {
+      // lookup by name
       final String instanceName = info.getInstanceName();
-      instanceId = getInstanceID(zooCache, instanceName);
-      verifyInstanceId(zooCache, instanceId.canonical(), instanceName);
+      String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
+      byte[] data = zooCache.get(instanceNamePath);
+      if (data == null) {
+        throw new RuntimeException(
+            "Instance name " + instanceName + " does not exist in zookeeper. "
+                + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
+      }
+      String instanceIdString = new String(data, UTF_8);
+      // verify that the instanceId found via the instanceName actually exists as an instance
+      if (zooCache.get(Constants.ZROOT + "/" + instanceIdString) == null) {
+        throw new RuntimeException("Instance id " + instanceIdString
+            + (instanceName == null ? "" : " pointed to by the name " + instanceName)
+            + " does not exist in zookeeper");
+      }
+      instanceId = InstanceId.of(instanceIdString);
     }
     return instanceId;
   }
 
-  // available only for sharing code with old ZooKeeperInstance
-  public static InstanceId getInstanceID(ZooCache zooCache, String instanceName) {
-    requireNonNull(zooCache, "zooCache cannot be null");
-    requireNonNull(instanceName, "instanceName cannot be null");
-    String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
-    byte[] data = zooCache.get(instanceNamePath);
-    if (data == null) {
-      throw new RuntimeException("Instance name " + instanceName + " does not exist in zookeeper. "
-          + "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
-    }
-    return InstanceId.of(new String(data, UTF_8));
-  }
-
-  // available only for sharing code with old ZooKeeperInstance
-  public static void verifyInstanceId(ZooCache zooCache, String instanceId, String instanceName) {
-    requireNonNull(zooCache, "zooCache cannot be null");
-    requireNonNull(instanceId, "instanceId cannot be null");
-    if (zooCache.get(Constants.ZROOT + "/" + instanceId) == null) {
-      throw new RuntimeException("Instance id " + instanceId
-          + (instanceName == null ? "" : " pointed to by the name " + instanceName)
-          + " does not exist in zookeeper");
-    }
-  }
-
   public String getZooKeeperRoot() {
     ensureOpen();
     return ZooUtil.getRoot(getInstanceID());
@@ -849,18 +837,6 @@
   }
 
   @Override
-  @Deprecated
-  public synchronized org.apache.accumulo.core.client.admin.ReplicationOperations
-      replicationOperations() {
-    ensureOpen();
-    if (replicationops == null) {
-      replicationops = new ReplicationOperationsImpl(this);
-    }
-
-    return replicationops;
-  }
-
-  @Override
   public Properties properties() {
     ensureOpen();
     Properties result = new Properties();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
index ad7827c..699fa49 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import java.io.IOException;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -62,15 +60,14 @@
   }
 
   @Override
-  public <T> T instantiate(String className, Class<T> base)
-      throws ReflectiveOperationException, IOException {
+  public <T> T instantiate(String className, Class<T> base) throws ReflectiveOperationException {
     return ClientServiceEnvironmentImpl.class.getClassLoader().loadClass(className).asSubclass(base)
         .getDeclaredConstructor().newInstance();
   }
 
   @Override
   public <T> T instantiate(TableId tableId, String className, Class<T> base)
-      throws ReflectiveOperationException, IOException {
+      throws ReflectiveOperationException {
     return instantiate(className, base);
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java
deleted file mode 100644
index f09adc2..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/CompactionStrategyConfigUtil.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.util.Map;
-
-import org.apache.accumulo.core.client.admin.CompactionConfig;
-import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
-
-@SuppressWarnings("removal")
-public class CompactionStrategyConfigUtil {
-
-  public static final CompactionStrategyConfig DEFAULT_STRATEGY = new CompactionStrategyConfig("") {
-    @Override
-    public CompactionStrategyConfig setOptions(Map<String,String> opts) {
-      throw new UnsupportedOperationException();
-    }
-  };
-
-  private static final int MAGIC = 0xcc5e6024;
-
-  public static void encode(DataOutput dout, CompactionConfig cc) {
-    var cs = cc.getCompactionStrategy();
-    UserCompactionUtils.encode(dout, MAGIC, 1, cs.getClassName(), cs.getOptions());
-  }
-
-  public static void decode(CompactionConfig cc, DataInput din) {
-    var pcd = UserCompactionUtils.decode(din, MAGIC, 1);
-    var csc = new CompactionStrategyConfig(pcd.className).setOptions(pcd.opts);
-
-    if (!isDefault(csc)) {
-      cc.setCompactionStrategy(csc);
-    }
-  }
-
-  public static boolean isDefault(CompactionStrategyConfig compactionStrategy) {
-    return compactionStrategy.equals(DEFAULT_STRATEGY);
-
-  }
-
-  public static boolean isDefault(CompactionConfig compactionConfig) {
-    return isDefault(compactionConfig.getCompactionStrategy());
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
index cb76751..654135a 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -42,6 +42,8 @@
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.access.AccessEvaluator;
+import org.apache.accumulo.access.InvalidAccessExpressionException;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -50,6 +52,7 @@
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Condition;
@@ -63,21 +66,15 @@
 import org.apache.accumulo.core.dataImpl.thrift.TConditionalSession;
 import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.commons.collections4.map.LRUMap;
@@ -91,6 +88,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
 
 public class ConditionalWriterImpl implements ConditionalWriter {
 
@@ -98,14 +96,14 @@
 
   private static final int MAX_SLEEP = 30000;
 
-  private Authorizations auths;
-  private VisibilityEvaluator ve;
-  private Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap<>(1000));
+  private final Authorizations auths;
+  private final AccessEvaluator accessEvaluator;
+  private final Map<Text,Boolean> cache = Collections.synchronizedMap(new LRUMap<>(1000));
   private final ClientContext context;
-  private TabletLocator locator;
+  private final TabletLocator locator;
   private final TableId tableId;
   private final String tableName;
-  private long timeout;
+  private final long timeout;
   private final Durability durability;
   private final String classLoaderContext;
   private final ConditionalWriterConfig config;
@@ -154,7 +152,7 @@
         count--;
         return result;
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
 
@@ -233,7 +231,7 @@
 
     @Override
     public void run() {
-      TabletClientService.Iface client = null;
+      TabletIngestClientService.Iface client = null;
 
       for (SessionID sid : sessions) {
         if (!sid.isActive()) {
@@ -377,7 +375,7 @@
     this.config = config;
     this.context = context;
     this.auths = config.getAuthorizations();
-    this.ve = new VisibilityEvaluator(config.getAuthorizations());
+    this.accessEvaluator = AccessEvaluator.of(config.getAuthorizations().toAccessAuthorizations());
     this.threadPool = context.threadPools().createScheduledExecutorService(
         config.getMaxWriteThreads(), this.getClass().getSimpleName());
     this.locator = new SyncingTabletLocator(context, tableId);
@@ -486,7 +484,7 @@
 
   private HashMap<HostAndPort,SessionID> cachedSessionIDs = new HashMap<>();
 
-  private SessionID reserveSessionID(HostAndPort location, TabletClientService.Iface client,
+  private SessionID reserveSessionID(HostAndPort location, TabletIngestClientService.Iface client,
       TInfo tinfo) throws ThriftSecurityException, TException {
     // avoid cost of repeatedly making RPC to create sessions, reuse sessions
     synchronized (cachedSessionIDs) {
@@ -555,18 +553,19 @@
     return activeSessions;
   }
 
-  private TabletClientService.Iface getClient(HostAndPort location) throws TTransportException {
-    TabletClientService.Iface client;
+  private TabletIngestClientService.Iface getClient(HostAndPort location)
+      throws TTransportException {
+    TabletIngestClientService.Iface client;
     if (timeout < context.getClientTimeoutInMillis()) {
-      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context, timeout);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context, timeout);
     } else {
-      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, location, context);
+      client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, location, context);
     }
     return client;
   }
 
   private void sendToServer(HostAndPort location, TabletServerMutations<QCMutation> mutations) {
-    TabletClientService.Iface client = null;
+    TabletIngestClientService.Iface client = null;
 
     TInfo tinfo = TraceUtil.traceInfo();
 
@@ -723,7 +722,7 @@
   }
 
   private void invalidateSession(long sessionId, HostAndPort location) throws TException {
-    TabletClientService.Iface client = null;
+    TabletIngestClientService.Iface client = null;
 
     TInfo tinfo = TraceUtil.traceInfo();
 
@@ -810,21 +809,24 @@
   }
 
   private boolean isVisible(ByteSequence cv) {
-    Text testVis = new Text(cv.toArray());
-    if (testVis.getLength() == 0) {
+
+    if (cv.length() == 0) {
       return true;
     }
 
+    byte[] arrayVis = cv.toArray();
+    Text testVis = new Text(arrayVis);
+
     Boolean b = cache.get(testVis);
     if (b != null) {
       return b;
     }
 
     try {
-      boolean bb = ve.evaluate(new ColumnVisibility(testVis));
+      boolean bb = accessEvaluator.canAccess(arrayVis);
       cache.put(new Text(testVis), bb);
       return bb;
-    } catch (VisibilityParseException | BadArgumentException e) {
+    } catch (InvalidAccessExpressionException e) {
       return false;
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
deleted file mode 100644
index e55375a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ConnectorImpl.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-
-import java.util.List;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchDeleter;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ConditionalWriter;
-import org.apache.accumulo.core.client.ConditionalWriterConfig;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.InstanceOperations;
-import org.apache.accumulo.core.client.admin.NamespaceOperations;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.singletons.SingletonManager;
-import org.apache.accumulo.core.singletons.SingletonManager.Mode;
-import org.apache.accumulo.core.trace.TraceUtil;
-
-/**
- * This class now delegates to {@link ClientContext}, except for the methods which were not copied
- * over to that.
- */
-@Deprecated(since = "2.0.0")
-public class ConnectorImpl extends org.apache.accumulo.core.client.Connector {
-
-  private static final String SYSTEM_TOKEN_NAME =
-      "org.apache.accumulo.server.security.SystemCredentials$SystemToken";
-  private final ClientContext context;
-
-  public ConnectorImpl(ClientContext context) throws AccumuloSecurityException, AccumuloException {
-    this.context = context;
-    SingletonManager.setMode(Mode.CONNECTOR);
-    if (context.getCredentials().getToken().isDestroyed()) {
-      throw new AccumuloSecurityException(context.getCredentials().getPrincipal(),
-          SecurityErrorCode.TOKEN_EXPIRED);
-    }
-    // Skip fail fast for system services; string literal for class name, to avoid dependency on
-    // server jar
-    final String tokenClassName = context.getCredentials().getToken().getClass().getName();
-    if (!SYSTEM_TOKEN_NAME.equals(tokenClassName)) {
-      if (!ThriftClientTypes.CLIENT.execute(context,
-          client -> client.authenticate(TraceUtil.traceInfo(), context.rpcCreds()))) {
-        throw new AccumuloSecurityException("Authentication failed, access denied",
-            SecurityErrorCode.BAD_CREDENTIALS);
-      }
-    }
-  }
-
-  public ClientContext getAccumuloClient() {
-    return context;
-  }
-
-  @Override
-  public org.apache.accumulo.core.client.Instance getInstance() {
-    return new org.apache.accumulo.core.client.Instance() {
-      @Override
-      public String getRootTabletLocation() {
-        return context.getRootTabletLocation();
-      }
-
-      @Override
-      public List<String> getMasterLocations() {
-        return context.getManagerLocations();
-      }
-
-      @Override
-      public String getInstanceID() {
-        return context.getInstanceID().canonical();
-      }
-
-      @Override
-      public String getInstanceName() {
-        return context.getInstanceName();
-      }
-
-      @Override
-      public String getZooKeepers() {
-        return context.getZooKeepers();
-      }
-
-      @Override
-      public int getZooKeepersSessionTimeOut() {
-        return context.getZooKeepersSessionTimeOut();
-      }
-
-      @Override
-      public org.apache.accumulo.core.client.Connector getConnector(String principal,
-          AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
-        return org.apache.accumulo.core.client.Connector.from(context);
-      }
-    };
-  }
-
-  @Override
-  public BatchScanner createBatchScanner(String tableName, Authorizations authorizations,
-      int numQueryThreads) throws TableNotFoundException {
-    return context.createBatchScanner(tableName, authorizations, numQueryThreads);
-  }
-
-  @Override
-  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, long maxMemory, long maxLatency, int maxWriteThreads)
-      throws TableNotFoundException {
-    return context.createBatchDeleter(tableName, authorizations, numQueryThreads,
-        new BatchWriterConfig().setMaxMemory(maxMemory).setMaxLatency(maxLatency, MILLISECONDS)
-            .setMaxWriteThreads(maxWriteThreads));
-  }
-
-  @Override
-  public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations,
-      int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException {
-    return context.createBatchDeleter(tableName, authorizations, numQueryThreads, config);
-  }
-
-  @Override
-  public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency,
-      int maxWriteThreads) throws TableNotFoundException {
-    return context.createBatchWriter(tableName, new BatchWriterConfig().setMaxMemory(maxMemory)
-        .setMaxLatency(maxLatency, MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
-  @Override
-  public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config)
-      throws TableNotFoundException {
-    return context.createBatchWriter(tableName, config);
-  }
-
-  @Override
-  public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency,
-      int maxWriteThreads) {
-    return context.createMultiTableBatchWriter(new BatchWriterConfig().setMaxMemory(maxMemory)
-        .setMaxLatency(maxLatency, MILLISECONDS).setMaxWriteThreads(maxWriteThreads));
-  }
-
-  @Override
-  public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) {
-    return context.createMultiTableBatchWriter(config);
-  }
-
-  @Override
-  public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config)
-      throws TableNotFoundException {
-    return context.createConditionalWriter(tableName, config);
-  }
-
-  @Override
-  public Scanner createScanner(String tableName, Authorizations authorizations)
-      throws TableNotFoundException {
-    return context.createScanner(tableName, authorizations);
-  }
-
-  @Override
-  public String whoami() {
-    return context.whoami();
-  }
-
-  @Override
-  public TableOperations tableOperations() {
-    return context.tableOperations();
-  }
-
-  @Override
-  public NamespaceOperations namespaceOperations() {
-    return context.namespaceOperations();
-  }
-
-  @Override
-  public SecurityOperations securityOperations() {
-    return context.securityOperations();
-  }
-
-  @Override
-  public InstanceOperations instanceOperations() {
-    return context.instanceOperations();
-  }
-
-  @Override
-  public ReplicationOperations replicationOperations() {
-    return context.replicationOperations();
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
index e32047b..eb7dad6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/Credentials.java
@@ -86,14 +86,14 @@
    *
    * @param instanceID Accumulo instance ID
    * @return Thrift credentials
-   * @throws RuntimeException if the authentication token has been destroyed (expired)
+   * @throws IllegalStateException if the authentication token has been destroyed (expired)
    */
   public TCredentials toThrift(InstanceId instanceID) {
     TCredentials tCreds = new TCredentials(getPrincipal(), getToken().getClass().getName(),
         ByteBuffer.wrap(AuthenticationTokenSerializer.serialize(getToken())),
         instanceID.canonical());
     if (getToken().isDestroyed()) {
-      throw new RuntimeException("Token has been destroyed",
+      throw new IllegalStateException("Token has been destroyed",
           new AccumuloSecurityException(getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED));
     }
     return tCreds;
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
index f59f9f3..dd7ce09 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/DurabilityImpl.java
@@ -19,7 +19,7 @@
 package org.apache.accumulo.core.clientImpl;
 
 import org.apache.accumulo.core.client.Durability;
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
+import org.apache.accumulo.core.tabletingest.thrift.TDurability;
 
 public class DurabilityImpl {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index 084d59e..215f7c6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -20,15 +20,13 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toList;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.createTransport;
 import static org.apache.accumulo.core.rpc.ThriftUtil.getClient;
 import static org.apache.accumulo.core.rpc.ThriftUtil.returnClient;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
@@ -56,11 +54,10 @@
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
 import org.apache.accumulo.core.util.Retry;
@@ -69,6 +66,8 @@
 import org.apache.thrift.transport.TTransport;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 /**
  * Provides a class for administering the accumulo instance
  */
@@ -137,9 +136,9 @@
 
     var log = LoggerFactory.getLogger(InstanceOperationsImpl.class);
 
-    Retry retry =
-        Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
-            .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (true) {
       try {
@@ -206,6 +205,13 @@
   }
 
   @Override
+  public Map<String,String> getSystemProperties()
+      throws AccumuloException, AccumuloSecurityException {
+    return ThriftClientTypes.CLIENT.execute(context,
+        client -> client.getSystemProperties(TraceUtil.traceInfo(), context.rpcCreds()));
+  }
+
+  @Override
   public List<String> getManagerLocations() {
     return context.getManagerLocations();
   }
@@ -297,7 +303,7 @@
   public List<ActiveCompaction> getActiveCompactions()
       throws AccumuloException, AccumuloSecurityException {
 
-    Map<String,List<HostAndPort>> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    Map<String,Set<HostAndPort>> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
     List<String> tservers = getTabletServers();
 
     int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
@@ -344,8 +350,7 @@
 
   @Override
   public void ping(String tserver) throws AccumuloException {
-    try (
-        TTransport transport = createTransport(AddressUtil.parseAddress(tserver, false), context)) {
+    try (TTransport transport = createTransport(AddressUtil.parseAddress(tserver), context)) {
       Client client = createClient(ThriftClientTypes.TABLET_SERVER, transport);
       client.getTabletServerStatus(TraceUtil.traceInfo(), context.rpcCreds());
     } catch (TException e) {
@@ -360,7 +365,7 @@
           client -> client.waitForBalance(TraceUtil.traceInfo()));
     } catch (AccumuloSecurityException ex) {
       // should never happen
-      throw new RuntimeException("Unexpected exception thrown", ex);
+      throw new IllegalStateException("Unexpected exception thrown", ex);
     }
 
   }
@@ -382,12 +387,6 @@
   }
 
   @Override
-  @Deprecated(since = "2.1.0")
-  public String getInstanceID() {
-    return getInstanceId().canonical();
-  }
-
-  @Override
   public InstanceId getInstanceId() {
     return context.getInstanceID();
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
index 0b1d496..3c92fa5 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/NamespaceOperationsImpl.java
@@ -20,13 +20,12 @@
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.util.Validators.EXISTING_NAMESPACE_NAME;
 import static org.apache.accumulo.core.util.Validators.NEW_NAMESPACE_NAME;
 
 import java.nio.ByteBuffer;
+import java.time.Duration;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
@@ -245,9 +244,9 @@
     EXISTING_NAMESPACE_NAME.validate(namespace);
     checkArgument(mapMutator != null, "mapMutator is null");
 
-    Retry retry =
-        Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
-            .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (true) {
       try {
@@ -338,7 +337,8 @@
   public Map<String,String> namespaceIdMap() {
     return Namespaces.getNameToIdMap(context).entrySet().stream()
         .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
-          throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
+          throw new IllegalStateException(
+              String.format("Duplicate key for values %s and %s", v1, v2));
         }, TreeMap::new));
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
index a03cc81..9200591 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/OfflineIterator.java
@@ -18,13 +18,14 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -59,14 +60,12 @@
 import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
 import org.apache.accumulo.core.manager.state.tables.TableState;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
-import org.apache.accumulo.core.metadata.TabletFile;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.hadoop.conf.Configuration;
@@ -94,12 +93,6 @@
       this.sampleConf = samplerConf;
     }
 
-    @Deprecated(since = "2.0.0")
-    @Override
-    public AccumuloConfiguration getConfig() {
-      return conf;
-    }
-
     @Override
     public IteratorScope getIteratorScope() {
       return IteratorScope.scan;
@@ -118,12 +111,6 @@
     private final ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators =
         new ArrayList<>();
 
-    @Deprecated(since = "2.0.0")
-    @Override
-    public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
-      topLevelIterators.add(iter);
-    }
-
     @Override
     public Authorizations getAuthorizations() {
       return authorizations;
@@ -157,12 +144,6 @@
           sampleConf);
     }
 
-    @Deprecated(since = "2.1.0")
-    @Override
-    public ServiceEnvironment getServiceEnv() {
-      return new ClientServiceEnvironmentImpl(context);
-    }
-
     @Override
     public PluginEnvironment getPluginEnv() {
       return new ClientServiceEnvironmentImpl(context);
@@ -205,11 +186,10 @@
         nextTablet();
       }
 
-    } catch (Exception e) {
-      if (e instanceof RuntimeException) {
-        throw (RuntimeException) e;
-      }
-      throw new RuntimeException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -232,8 +212,10 @@
       }
 
       return ret;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+      throw new IllegalStateException(e);
     }
   }
 
@@ -329,11 +311,11 @@
     if (scannerSamplerConfigImpl != null && !scannerSamplerConfigImpl.equals(samplerConfImpl)) {
       throw new SampleNotPresentException();
     }
-    for (TabletFile file : absFiles) {
+    for (StoredTabletFile file : absFiles) {
       var cs = CryptoFactoryLoader.getServiceForClientWithTable(systemConf, tableConf, tableId);
-      FileSystem fs = VolumeConfiguration.fileSystemForPath(file.getPathStr(), conf);
+      FileSystem fs = VolumeConfiguration.fileSystemForPath(file.getNormalizedPathStr(), conf);
       FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-          .forFile(file.getPathStr(), fs, conf, cs).withTableConfiguration(tableCC).build();
+          .forFile(file, fs, conf, cs).withTableConfiguration(tableCC).build();
       if (scannerSamplerConfigImpl != null) {
         reader = reader.getSample(scannerSamplerConfigImpl);
         if (reader == null) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
deleted file mode 100644
index 49aa1f5..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static java.util.Objects.requireNonNull;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.ReplicationOperations;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Deprecated
-public class ReplicationOperationsImpl implements ReplicationOperations {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImpl.class);
-
-  private final ClientContext context;
-
-  public ReplicationOperationsImpl(ClientContext context) {
-    this.context = requireNonNull(context);
-  }
-
-  @Override
-  public void addPeer(final String name, final String replicaType)
-      throws AccumuloException, AccumuloSecurityException {
-    context.instanceOperations().setProperty(
-        Property.REPLICATION_PEERS.getKey() + requireNonNull(name), requireNonNull(replicaType));
-  }
-
-  @Override
-  public void removePeer(final String name) throws AccumuloException, AccumuloSecurityException {
-    context.instanceOperations()
-        .removeProperty(Property.REPLICATION_PEERS.getKey() + requireNonNull(name));
-  }
-
-  @Override
-  public void drain(String tableName)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    drain(tableName, referencedFiles(requireNonNull(tableName)));
-  }
-
-  @Override
-  public void drain(final String tableName, final Set<String> wals)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    requireNonNull(tableName);
-
-    final TInfo tinfo = TraceUtil.traceInfo();
-    final TCredentials rpcCreds = context.rpcCreds();
-
-    // Ask the manager if the table is fully replicated given these WALs, but don't poll inside the
-    // manager
-    boolean drained = false;
-    while (!drained) {
-      drained = getManagerDrain(tinfo, rpcCreds, tableName, wals);
-
-      if (!drained) {
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException("Thread interrupted", e);
-        }
-      }
-    }
-  }
-
-  protected boolean getManagerDrain(final TInfo tinfo, final TCredentials rpcCreds,
-      final String tableName, final Set<String> wals)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    return ThriftClientTypes.MANAGER.execute(context,
-        client -> client.drainReplicationTable(tinfo, rpcCreds, tableName, wals));
-  }
-
-  @Override
-  public Set<String> referencedFiles(String tableName) throws TableNotFoundException {
-    log.debug("Collecting referenced files for replication of table {}", tableName);
-    TableId tableId = context.getTableId(tableName);
-    log.debug("Found id of {} for name {}", tableId, tableName);
-
-    // Get the WALs currently referenced by the table
-    BatchScanner metaBs = context.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    metaBs.setRanges(Collections.singleton(TabletsSection.getRange(tableId)));
-    metaBs.fetchColumnFamily(LogColumnFamily.NAME);
-    Set<String> wals = new HashSet<>();
-    try {
-      for (Entry<Key,Value> entry : metaBs) {
-        LogEntry logEntry = LogEntry.fromMetaWalEntry(entry);
-        wals.add(new Path(logEntry.filename).toString());
-      }
-    } finally {
-      metaBs.close();
-    }
-
-    // And the WALs that need to be replicated for this table
-    metaBs = context.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    metaBs.setRanges(Collections.singleton(ReplicationSection.getRange()));
-    metaBs.fetchColumnFamily(ReplicationSection.COLF);
-    try {
-      Text buffer = new Text();
-      for (Entry<Key,Value> entry : metaBs) {
-        if (tableId.equals(ReplicationSection.getTableId(entry.getKey()))) {
-          ReplicationSection.getFile(entry.getKey(), buffer);
-          wals.add(buffer.toString());
-        }
-      }
-    } finally {
-      metaBs.close();
-    }
-    return wals;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
index 28aa5b9..a24a236 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/RootTabletLocator.java
@@ -18,10 +18,10 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -56,11 +56,11 @@
       Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures) {
     TabletLocation rootTabletLocation = getRootTabletLocation(context);
     if (rootTabletLocation != null) {
-      TabletServerMutations<T> tsm = new TabletServerMutations<>(rootTabletLocation.tablet_session);
+      var tsm = new TabletServerMutations<T>(rootTabletLocation.getTserverSession());
       for (T mutation : mutations) {
         tsm.addMutation(RootTable.EXTENT, mutation);
       }
-      binnedMutations.put(rootTabletLocation.tablet_location, tsm);
+      binnedMutations.put(rootTabletLocation.getTserverLocation(), tsm);
     } else {
       failures.addAll(mutations);
     }
@@ -73,7 +73,7 @@
     TabletLocation rootTabletLocation = getRootTabletLocation(context);
     if (rootTabletLocation != null) {
       for (Range range : ranges) {
-        TabletLocatorImpl.addRange(binnedRanges, rootTabletLocation.tablet_location,
+        TabletLocatorImpl.addRange(binnedRanges, rootTabletLocation.getTserverLocation(),
             RootTable.EXTENT, range);
       }
       return Collections.emptyList();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
index 182b912..1f065bb 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
@@ -27,11 +27,15 @@
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.SampleNotPresentException;
 import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
 import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.clientImpl.ThriftScanner.ScanState;
+import org.apache.accumulo.core.clientImpl.ThriftScanner.ScanTimedOutException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.Range;
@@ -143,7 +147,8 @@
     readAheadOperation = context.submitScannerReadAheadTask(this::readBatch);
   }
 
-  private List<KeyValue> readBatch() throws Exception {
+  private List<KeyValue> readBatch() throws ScanTimedOutException, AccumuloException,
+      AccumuloSecurityException, TableNotFoundException {
 
     List<KeyValue> batch;
 
@@ -176,11 +181,10 @@
       }
     } catch (ExecutionException ee) {
       wrapExecutionException(ee);
-      throw new RuntimeException(ee);
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(ee);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException
+        | ScanTimedOutException | InterruptedException e) {
+      throw new IllegalStateException(e);
     }
 
     if (!nextBatch.isEmpty()) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
index 8f6daed..df40a21 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/SyncingTabletLocator.java
@@ -21,7 +21,7 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
+import java.util.function.Supplier;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -31,8 +31,6 @@
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Syncs itself with the static collection of TabletLocators, so that when the server clears it, it
@@ -40,19 +38,13 @@
  * using SyncingTabletLocator.
  */
 public class SyncingTabletLocator extends TabletLocator {
-  private static final Logger log = LoggerFactory.getLogger(SyncingTabletLocator.class);
 
   private volatile TabletLocator locator;
-  private final Callable<TabletLocator> getLocatorFunction;
+  private final Supplier<TabletLocator> getLocatorFunction;
 
-  public SyncingTabletLocator(Callable<TabletLocator> getLocatorFunction) {
+  public SyncingTabletLocator(Supplier<TabletLocator> getLocatorFunction) {
     this.getLocatorFunction = getLocatorFunction;
-    try {
-      this.locator = getLocatorFunction.call();
-    } catch (Exception e) {
-      log.error("Problem obtaining TabletLocator", e);
-      throw new RuntimeException(e);
-    }
+    this.locator = getLocatorFunction.get();
   }
 
   public SyncingTabletLocator(final ClientContext context, final TableId tableId) {
@@ -64,12 +56,7 @@
     if (!loc.isValid()) {
       synchronized (this) {
         if (locator == loc) {
-          try {
-            loc = locator = getLocatorFunction.call();
-          } catch (Exception e) {
-            log.error("Problem obtaining TabletLocator", e);
-            throw new RuntimeException(e);
-          }
+          loc = locator = getLocatorFunction.get();
         }
       }
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index 129cb6a..6b94fd8 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -19,15 +19,15 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toSet;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
 import static org.apache.accumulo.core.util.Validators.NEW_TABLE_NAME;
 
@@ -36,7 +36,7 @@
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
-import java.security.SecureRandom;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -119,9 +119,8 @@
 import org.apache.accumulo.core.manager.thrift.FateOperation;
 import org.apache.accumulo.core.manager.thrift.FateService;
 import org.apache.accumulo.core.manager.thrift.ManagerClientService;
-import org.apache.accumulo.core.metadata.MetadataServicer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.AccumuloTable;
+import org.apache.accumulo.core.metadata.schema.TabletDeletedException;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
@@ -132,10 +131,9 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.summary.SummarizerConfigurationUtil;
 import org.apache.accumulo.core.summary.SummaryCollection;
+import org.apache.accumulo.core.tablet.thrift.TabletManagementClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
 import org.apache.accumulo.core.util.MapCounter;
@@ -156,11 +154,10 @@
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 public class TableOperationsImpl extends TableOperationsHelper {
 
-  private static final SecureRandom random = new SecureRandom();
-
   public static final String PROPERTY_EXCLUDE_PREFIX = "!";
   public static final String COMPACTION_CANCELED_MSG = "Compaction canceled";
   public static final String TABLE_DELETED_MSG = "Table is being deleted";
@@ -198,7 +195,8 @@
   public boolean exists(String tableName) {
     EXISTING_TABLE_NAME.validate(tableName);
 
-    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
+    if (tableName.equals(AccumuloTable.METADATA.tableName())
+        || tableName.equals(AccumuloTable.ROOT.tableName())) {
       return true;
     }
 
@@ -531,7 +529,7 @@
         }
       }
     } catch (InterruptedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     } finally {
       executor.shutdown();
     }
@@ -562,26 +560,26 @@
           continue;
         }
 
-        HostAndPort address = HostAndPort.fromString(tl.tablet_location);
+        HostAndPort address = HostAndPort.fromString(tl.getTserverLocation());
 
         try {
-          TabletClientService.Client client =
-              ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, address, context);
+          TabletManagementClientService.Client client =
+              ThriftUtil.getClient(ThriftClientTypes.TABLET_MGMT, address, context);
           try {
 
             OpTimer timer = null;
 
             if (log.isTraceEnabled()) {
               log.trace("tid={} Splitting tablet {} on {} at {}", Thread.currentThread().getId(),
-                  tl.tablet_extent, address, split);
+                  tl.getExtent(), address, split);
               timer = new OpTimer().start();
             }
 
-            client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(),
-                tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
+            client.splitTablet(TraceUtil.traceInfo(), context.rpcCreds(), tl.getExtent().toThrift(),
+                TextUtil.getByteBuffer(split));
 
             // just split it, might as well invalidate it in the cache
-            tabLocator.invalidateCache(tl.tablet_extent);
+            tabLocator.invalidateCache(tl.getExtent());
 
             if (timer != null) {
               timer.stop();
@@ -606,10 +604,10 @@
                 + " Seen {} failures.", split, env.tableName, locationFailures);
           }
 
-          tabLocator.invalidateCache(tl.tablet_extent);
+          tabLocator.invalidateCache(tl.getExtent());
           continue;
         } catch (TException e) {
-          tabLocator.invalidateCache(context, tl.tablet_location);
+          tabLocator.invalidateCache(context, tl.getTserverLocation());
           continue;
         }
 
@@ -665,37 +663,22 @@
 
   private List<Text> _listSplits(String tableName)
       throws TableNotFoundException, AccumuloSecurityException {
+
     TableId tableId = context.getTableId(tableName);
-    TreeMap<KeyExtent,String> tabletLocations = new TreeMap<>();
+
     while (true) {
-      try {
-        tabletLocations.clear();
-        // the following method throws AccumuloException for some conditions that should be retried
-        MetadataServicer.forTableId(context, tableId).getTabletLocations(tabletLocations);
-        break;
-      } catch (AccumuloSecurityException ase) {
-        throw ase;
-      } catch (Exception e) {
+      try (TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId)
+          .fetch(PREV_ROW).checkConsistency().build()) {
+        return tabletsMetadata.stream().map(tm -> tm.getExtent().endRow()).filter(Objects::nonNull)
+            .collect(Collectors.toList());
+      } catch (TabletDeletedException tde) {
+        // see if the table was deleted
         context.requireTableExists(tableId, tableName);
-
-        if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
-          throw (AccumuloSecurityException) e.getCause();
-        }
-
-        log.info("{} ... retrying ...", e, e);
+        log.debug("A merge happened while trying to list splits for {} {}, retrying ", tableName,
+            tableId, tde);
         sleepUninterruptibly(3, SECONDS);
       }
     }
-
-    ArrayList<Text> endRows = new ArrayList<>(tabletLocations.size());
-    for (KeyExtent ke : tabletLocations.keySet()) {
-      if (ke.endRow() != null) {
-        endRows.add(ke.endRow());
-      }
-    }
-
-    return endRows;
-
   }
 
   /**
@@ -860,8 +843,6 @@
       }
     }
 
-    ensureStrategyCanLoad(tableName, config);
-
     if (!UserCompactionUtils.isDefault(config.getConfigurer())) {
       if (!testClassLoad(tableName, config.getConfigurer().getClassName(),
           CompactionConfigurer.class.getName())) {
@@ -902,19 +883,6 @@
     }
   }
 
-  @SuppressWarnings("removal")
-  private void ensureStrategyCanLoad(String tableName, CompactionConfig config)
-      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    // Make sure the specified compaction strategy exists on a tabletserver
-    if (!CompactionStrategyConfigUtil.isDefault(config.getCompactionStrategy())) {
-      if (!testClassLoad(tableName, config.getCompactionStrategy().getClassName(),
-          "org.apache.accumulo.tserver.compaction.CompactionStrategy")) {
-        throw new AccumuloException("TabletServer could not load CompactionStrategy class "
-            + config.getCompactionStrategy().getClassName());
-      }
-    }
-  }
-
   @Override
   public void cancelCompaction(String tableName)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
@@ -1053,9 +1021,9 @@
     EXISTING_TABLE_NAME.validate(tableName);
     checkArgument(mapMutator != null, "mapMutator is null");
 
-    Retry retry =
-        Retry.builder().infiniteRetries().retryAfter(25, MILLISECONDS).incrementBy(25, MILLISECONDS)
-            .maxWait(30, SECONDS).backOffFactor(1.5).logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        .incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (true) {
       try {
@@ -1261,7 +1229,7 @@
 
       log.warn("Unable to locate bins for specified range. Retrying.");
       // sleep randomly between 100 and 200ms
-      sleepUninterruptibly(100 + random.nextInt(100), MILLISECONDS);
+      sleepUninterruptibly(100 + RANDOM.get().nextInt(100), MILLISECONDS);
       binnedRanges.clear();
       tl.invalidateCache();
     }
@@ -1327,34 +1295,6 @@
     return ret;
   }
 
-  @Override
-  @Deprecated(since = "2.0.0")
-  public void importDirectory(String tableName, String dir, String failureDir, boolean setTime)
-      throws IOException, AccumuloSecurityException, TableNotFoundException, AccumuloException {
-    EXISTING_TABLE_NAME.validate(tableName);
-    checkArgument(dir != null, "dir is null");
-    checkArgument(failureDir != null, "failureDir is null");
-
-    // check for table existence
-    context.getTableId(tableName);
-    Path dirPath = checkPath(dir, "Bulk", "");
-    Path failPath = checkPath(failureDir, "Bulk", "failure");
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)),
-        ByteBuffer.wrap(dirPath.toString().getBytes(UTF_8)),
-        ByteBuffer.wrap(failPath.toString().getBytes(UTF_8)),
-        ByteBuffer.wrap((setTime + "").getBytes(UTF_8)));
-    Map<String,String> opts = new HashMap<>();
-
-    try {
-      doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT,
-          args, opts);
-    } catch (TableExistsException e) {
-      // should not happen
-      throw new AssertionError(e);
-    }
-  }
-
   private void waitForTableStateTransition(TableId tableId, TableState expectedState)
       throws AccumuloException, TableNotFoundException {
     Text startRow = null;
@@ -1381,9 +1321,6 @@
         range = new Range(startRow, lastRow);
       }
 
-      TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
-          .overRange(range).fetch(LOCATION, PREV_ROW).build();
-
       KeyExtent lastExtent = null;
 
       int total = 0;
@@ -1392,34 +1329,38 @@
       Text continueRow = null;
       MapCounter<String> serverCounts = new MapCounter<>();
 
-      for (TabletMetadata tablet : tablets) {
-        total++;
-        Location loc = tablet.getLocation();
+      try (TabletsMetadata tablets = TabletsMetadata.builder(context).scanMetadataTable()
+          .overRange(range).fetch(LOCATION, PREV_ROW).build()) {
 
-        if ((expectedState == TableState.ONLINE
-            && (loc == null || loc.getType() == LocationType.FUTURE))
-            || (expectedState == TableState.OFFLINE && loc != null)) {
-          if (continueRow == null) {
-            continueRow = tablet.getExtent().toMetaRow();
+        for (TabletMetadata tablet : tablets) {
+          total++;
+          Location loc = tablet.getLocation();
+
+          if ((expectedState == TableState.ONLINE
+              && (loc == null || loc.getType() == LocationType.FUTURE))
+              || (expectedState == TableState.OFFLINE && loc != null)) {
+            if (continueRow == null) {
+              continueRow = tablet.getExtent().toMetaRow();
+            }
+            waitFor++;
+            lastRow = tablet.getExtent().toMetaRow();
+
+            if (loc != null) {
+              serverCounts.increment(loc.getHostPortSession(), 1);
+            }
           }
-          waitFor++;
-          lastRow = tablet.getExtent().toMetaRow();
 
-          if (loc != null) {
-            serverCounts.increment(loc.getHostPortSession(), 1);
+          if (!tablet.getExtent().tableId().equals(tableId)) {
+            throw new AccumuloException(
+                "Saw unexpected table Id " + tableId + " " + tablet.getExtent());
           }
-        }
 
-        if (!tablet.getExtent().tableId().equals(tableId)) {
-          throw new AccumuloException(
-              "Saw unexpected table Id " + tableId + " " + tablet.getExtent());
-        }
+          if (lastExtent != null && !tablet.getExtent().isPreviousExtent(lastExtent)) {
+            holes++;
+          }
 
-        if (lastExtent != null && !tablet.getExtent().isPreviousExtent(lastExtent)) {
-          holes++;
+          lastExtent = tablet.getExtent();
         }
-
-        lastExtent = tablet.getExtent();
       }
 
       if (continueRow != null) {
@@ -1539,7 +1480,8 @@
   public Map<String,String> tableIdMap() {
     return context.getTableNameToIdMap().entrySet().stream()
         .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().canonical(), (v1, v2) -> {
-          throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
+          throw new IllegalStateException(
+              String.format("Duplicate key for values %s and %s", v1, v2));
         }, TreeMap::new));
   }
 
@@ -1563,7 +1505,7 @@
         // this operation may us a lot of memory... its likely that connections to tabletservers
         // hosting metadata tablets will be cached, so do not use cached
         // connections
-        pair = ThriftClientTypes.CLIENT.getTabletServerConnection(context, false);
+        pair = ThriftClientTypes.CLIENT.getThriftServerConnection(context, false);
         diskUsages = pair.getSecond().getDiskUsage(tableNames, context.rpcCreds());
       } catch (ThriftTableOperationException e) {
         switch (e.getType()) {
@@ -1886,7 +1828,7 @@
           List<Range> prev =
               groupedByTablets.put(tabletId, Collections.unmodifiableList(entry2.getValue()));
           if (prev != null) {
-            throw new RuntimeException(
+            throw new IllegalStateException(
                 "Unexpected : tablet at multiple locations : " + location + " " + tabletId);
           }
         }
@@ -1945,9 +1887,9 @@
 
     locator.invalidateCache();
 
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(2, SECONDS).backOffFactor(1.5)
-        .logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(2)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     while (!locator.binRanges(context, rangeList, binnedRanges).isEmpty()) {
       context.requireTableExists(tableId, tableName);
@@ -1958,7 +1900,7 @@
             String.format("locating tablets in table %s(%s) for %d ranges", tableName, tableId,
                 rangeList.size()));
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
       locator.invalidateCache();
     }
@@ -2119,8 +2061,11 @@
   @Override
   public TimeType getTimeType(final String tableName) throws TableNotFoundException {
     TableId tableId = context.getTableId(tableName);
-    Optional<TabletMetadata> tabletMetadata = context.getAmple().readTablets().forTable(tableId)
-        .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build().stream().findFirst();
+    Optional<TabletMetadata> tabletMetadata;
+    try (TabletsMetadata tabletsMetadata = context.getAmple().readTablets().forTable(tableId)
+        .fetch(TabletMetadata.ColumnType.TIME).checkConsistency().build()) {
+      tabletMetadata = tabletsMetadata.stream().findFirst();
+    }
     TabletMetadata timeData =
         tabletMetadata.orElseThrow(() -> new IllegalStateException("Failed to retrieve TimeType"));
     return timeData.getTime().getType();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
index b8e4c53..9c528c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocator.java
@@ -25,6 +25,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -34,9 +35,8 @@
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.MetadataLocationObtainer;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonService;
 import org.apache.accumulo.core.util.Interner;
@@ -140,14 +140,15 @@
     if (tl == null) {
       MetadataLocationObtainer mlo = new MetadataLocationObtainer();
 
-      if (RootTable.ID.equals(tableId)) {
+      if (AccumuloTable.ROOT.tableId().equals(tableId)) {
         tl = new RootTabletLocator(new ZookeeperLockChecker(context));
-      } else if (MetadataTable.ID.equals(tableId)) {
-        tl = new TabletLocatorImpl(MetadataTable.ID, getLocator(context, RootTable.ID), mlo,
+      } else if (AccumuloTable.METADATA.tableId().equals(tableId)) {
+        tl = new TabletLocatorImpl(AccumuloTable.METADATA.tableId(),
+            getLocator(context, AccumuloTable.ROOT.tableId()), mlo,
             new ZookeeperLockChecker(context));
       } else {
-        tl = new TabletLocatorImpl(tableId, getLocator(context, MetadataTable.ID), mlo,
-            new ZookeeperLockChecker(context));
+        tl = new TabletLocatorImpl(tableId, getLocator(context, AccumuloTable.METADATA.tableId()),
+            mlo, new ZookeeperLockChecker(context));
       }
       locators.put(key, tl);
     }
@@ -194,54 +195,53 @@
     }
   }
 
-  public static class TabletLocation implements Comparable<TabletLocation> {
+  public static class TabletLocation {
     private static final Interner<String> interner = new Interner<>();
 
-    public final KeyExtent tablet_extent;
-    public final String tablet_location;
-    public final String tablet_session;
+    private final KeyExtent tablet_extent;
+    private final String tserverLocation;
+    private final String tserverSession;
 
     public TabletLocation(KeyExtent tablet_extent, String tablet_location, String session) {
       checkArgument(tablet_extent != null, "tablet_extent is null");
       checkArgument(tablet_location != null, "tablet_location is null");
       checkArgument(session != null, "session is null");
       this.tablet_extent = tablet_extent;
-      this.tablet_location = interner.intern(tablet_location);
-      this.tablet_session = interner.intern(session);
+      this.tserverLocation = interner.intern(tablet_location);
+      this.tserverSession = interner.intern(session);
     }
 
     @Override
     public boolean equals(Object o) {
       if (o instanceof TabletLocation) {
         TabletLocation otl = (TabletLocation) o;
-        return tablet_extent.equals(otl.tablet_extent)
-            && tablet_location.equals(otl.tablet_location)
-            && tablet_session.equals(otl.tablet_session);
+        return getExtent().equals(otl.getExtent())
+            && getTserverLocation().equals(otl.getTserverLocation())
+            && getTserverSession().equals(otl.getTserverSession());
       }
       return false;
     }
 
     @Override
     public int hashCode() {
-      throw new UnsupportedOperationException(
-          "hashcode is not implemented for class " + this.getClass());
+      return Objects.hash(getExtent(), tserverLocation, tserverSession);
     }
 
     @Override
     public String toString() {
-      return "(" + tablet_extent + "," + tablet_location + "," + tablet_session + ")";
+      return "(" + getExtent() + "," + getTserverLocation() + "," + getTserverSession() + ")";
     }
 
-    @Override
-    public int compareTo(TabletLocation o) {
-      int result = tablet_extent.compareTo(o.tablet_extent);
-      if (result == 0) {
-        result = tablet_location.compareTo(o.tablet_location);
-        if (result == 0) {
-          result = tablet_session.compareTo(o.tablet_session);
-        }
-      }
-      return result;
+    public KeyExtent getExtent() {
+      return tablet_extent;
+    }
+
+    public String getTserverLocation() {
+      return tserverLocation;
+    }
+
+    public String getTserverSession() {
+      return tserverSession;
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
index 02844db..dafada9 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletLocatorImpl.java
@@ -18,9 +18,9 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.apache.accumulo.core.util.UtilWaitThread.sleepUninterruptibly;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -125,7 +125,7 @@
         return null;
       }
 
-      Pair<String,String> lock = new Pair<>(tl.tablet_location, tl.tablet_session);
+      Pair<String,String> lock = new Pair<>(tl.getTserverLocation(), tl.getTserverSession());
 
       if (okLocks.contains(lock)) {
         return tl;
@@ -135,14 +135,14 @@
         return null;
       }
 
-      if (lockChecker.isLockHeld(tl.tablet_location, tl.tablet_session)) {
+      if (lockChecker.isLockHeld(tl.getTserverLocation(), tl.getTserverSession())) {
         okLocks.add(lock);
         return tl;
       }
 
       if (log.isTraceEnabled()) {
-        log.trace("Tablet server {} {} no longer holds its lock", tl.tablet_location,
-            tl.tablet_session);
+        log.trace("Tablet server {} {} no longer holds its lock", tl.getTserverLocation(),
+            tl.getTserverSession());
       }
 
       invalidLocks.add(lock);
@@ -244,22 +244,22 @@
   private <T extends Mutation> boolean addMutation(
       Map<String,TabletServerMutations<T>> binnedMutations, T mutation, TabletLocation tl,
       LockCheckerSession lcSession) {
-    TabletServerMutations<T> tsm = binnedMutations.get(tl.tablet_location);
+    TabletServerMutations<T> tsm = binnedMutations.get(tl.getTserverLocation());
 
     if (tsm == null) {
       // do lock check once per tserver here to make binning faster
       boolean lockHeld = lcSession.checkLock(tl) != null;
       if (lockHeld) {
-        tsm = new TabletServerMutations<>(tl.tablet_session);
-        binnedMutations.put(tl.tablet_location, tsm);
+        tsm = new TabletServerMutations<>(tl.getTserverSession());
+        binnedMutations.put(tl.getTserverLocation(), tsm);
       } else {
         return false;
       }
     }
 
     // its possible the same tserver could be listed with different sessions
-    if (tsm.getSession().equals(tl.tablet_session)) {
-      tsm.addMutation(tl.tablet_extent, mutation);
+    if (tsm.getSession().equals(tl.getTserverSession())) {
+      tsm.addMutation(tl.getExtent(), mutation);
       return true;
     }
 
@@ -269,10 +269,10 @@
   static boolean isContiguous(List<TabletLocation> tabletLocations) {
 
     Iterator<TabletLocation> iter = tabletLocations.iterator();
-    KeyExtent prevExtent = iter.next().tablet_extent;
+    KeyExtent prevExtent = iter.next().getExtent();
 
     while (iter.hasNext()) {
-      KeyExtent currExtent = iter.next().tablet_extent;
+      KeyExtent currExtent = iter.next().getExtent();
 
       if (!currExtent.isPreviousExtent(prevExtent)) {
         return false;
@@ -323,14 +323,14 @@
 
       tabletLocations.add(tl);
 
-      while (tl.tablet_extent.endRow() != null
-          && !range.afterEndKey(new Key(tl.tablet_extent.endRow()).followingKey(PartialKey.ROW))) {
+      while (tl.getExtent().endRow() != null
+          && !range.afterEndKey(new Key(tl.getExtent().endRow()).followingKey(PartialKey.ROW))) {
         if (useCache) {
-          Text row = new Text(tl.tablet_extent.endRow());
+          Text row = new Text(tl.getExtent().endRow());
           row.append(new byte[] {0}, 0, 1);
           tl = lcSession.checkLock(locateTabletInCache(row));
         } else {
-          tl = _locateTablet(context, tl.tablet_extent.endRow(), true, false, false, lcSession);
+          tl = _locateTablet(context, tl.getExtent().endRow(), true, false, false, lcSession);
         }
 
         if (tl == null) {
@@ -349,7 +349,8 @@
       // then after that merges and splits happen.
       if (isContiguous(tabletLocations)) {
         for (TabletLocation tl2 : tabletLocations) {
-          TabletLocatorImpl.addRange(binnedRanges, tl2.tablet_location, tl2.tablet_extent, range);
+          TabletLocatorImpl.addRange(binnedRanges, tl2.getTserverLocation(), tl2.getExtent(),
+              range);
         }
       } else {
         failures.add(range);
@@ -454,8 +455,8 @@
     wLock.lock();
     try {
       for (TabletLocation cacheEntry : metaCache.values()) {
-        if (cacheEntry.tablet_location.equals(server)) {
-          badExtents.add(cacheEntry.tablet_extent);
+        if (cacheEntry.getTserverLocation().equals(server)) {
+          badExtents.add(cacheEntry.getExtent());
           invalidatedCount++;
         }
       }
@@ -516,7 +517,7 @@
       if (timer != null) {
         timer.stop();
         log.trace("tid={} Located tablet {} at {} in {}", Thread.currentThread().getId(),
-            (tl == null ? "null" : tl.tablet_extent), (tl == null ? "null" : tl.tablet_location),
+            (tl == null ? "null" : tl.getExtent()), (tl == null ? "null" : tl.getTserverLocation()),
             String.format("%.3f secs", timer.scale(SECONDS)));
       }
 
@@ -538,7 +539,7 @@
       while (locations != null && locations.getLocations().isEmpty()
           && locations.getLocationless().isEmpty()) {
         // try the next tablet, the current tablet does not have any tablets that overlap the row
-        Text er = ptl.tablet_extent.endRow();
+        Text er = ptl.getExtent().endRow();
         if (er != null && er.compareTo(lastTabletRow) < 0) {
           // System.out.println("er "+er+" ltr "+lastTabletRow);
           ptl = parent.locateTablet(context, er, true, retry);
@@ -563,20 +564,20 @@
       Text lastEndRow = null;
       for (TabletLocation tabletLocation : locations.getLocations()) {
 
-        KeyExtent ke = tabletLocation.tablet_extent;
+        KeyExtent ke = tabletLocation.getExtent();
         TabletLocation locToCache;
 
         // create new location if current prevEndRow == endRow
         if ((lastEndRow != null) && (ke.prevEndRow() != null)
             && ke.prevEndRow().equals(lastEndRow)) {
           locToCache = new TabletLocation(new KeyExtent(ke.tableId(), ke.endRow(), lastEndRow),
-              tabletLocation.tablet_location, tabletLocation.tablet_session);
+              tabletLocation.getTserverLocation(), tabletLocation.getTserverSession());
         } else {
           locToCache = tabletLocation;
         }
 
         // save endRow for next iteration
-        lastEndRow = locToCache.tablet_extent.endRow();
+        lastEndRow = locToCache.getExtent().endRow();
 
         updateCache(locToCache, lcSession);
       }
@@ -585,20 +586,20 @@
   }
 
   private void updateCache(TabletLocation tabletLocation, LockCheckerSession lcSession) {
-    if (!tabletLocation.tablet_extent.tableId().equals(tableId)) {
+    if (!tabletLocation.getExtent().tableId().equals(tableId)) {
       // sanity check
       throw new IllegalStateException(
-          "Unexpected extent returned " + tableId + "  " + tabletLocation.tablet_extent);
+          "Unexpected extent returned " + tableId + "  " + tabletLocation.getExtent());
     }
 
-    if (tabletLocation.tablet_location == null) {
+    if (tabletLocation.getTserverLocation() == null) {
       // sanity check
       throw new IllegalStateException(
-          "Cannot add null locations to cache " + tableId + "  " + tabletLocation.tablet_extent);
+          "Cannot add null locations to cache " + tableId + "  " + tabletLocation.getExtent());
     }
 
     // clear out any overlapping extents in cache
-    removeOverlapping(metaCache, tabletLocation.tablet_extent);
+    removeOverlapping(metaCache, tabletLocation.getExtent());
 
     // do not add to cache unless lock is held
     if (lcSession.checkLock(tabletLocation) == null) {
@@ -606,14 +607,14 @@
     }
 
     // add it to cache
-    Text er = tabletLocation.tablet_extent.endRow();
+    Text er = tabletLocation.getExtent().endRow();
     if (er == null) {
       er = MAX_TEXT;
     }
     metaCache.put(er, tabletLocation);
 
     if (!badExtents.isEmpty()) {
-      removeOverlapping(badExtents, tabletLocation.tablet_extent);
+      removeOverlapping(badExtents, tabletLocation.getExtent());
     }
   }
 
@@ -631,7 +632,7 @@
     while (iter.hasNext()) {
       Entry<Text,TabletLocation> entry = iter.next();
 
-      KeyExtent ke = entry.getValue().tablet_extent;
+      KeyExtent ke = entry.getValue().getExtent();
 
       if (stopRemoving(nke, ke)) {
         break;
@@ -663,7 +664,7 @@
     Entry<Text,TabletLocation> entry = metaCache.ceilingEntry(row);
 
     if (entry != null) {
-      KeyExtent ke = entry.getValue().tablet_extent;
+      KeyExtent ke = entry.getValue().getExtent();
       if (ke.prevEndRow() == null || ke.prevEndRow().compareTo(row) < 0) {
         return entry.getValue();
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
index 322fd0f..b39240f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.core.clientImpl;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.io.IOException;
@@ -73,13 +71,12 @@
 import org.apache.accumulo.core.spi.scan.ScanServerAttempt;
 import org.apache.accumulo.core.spi.scan.ScanServerSelections;
 import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
-import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.thrift.TApplicationException;
@@ -88,6 +85,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value>> {
 
   private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReaderIterator.class);
@@ -160,17 +159,15 @@
           log.warn("Failed to add Batch Scan result", e);
         }
         fatalException = e;
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
 
       }
     };
 
     try {
       lookup(ranges, rr);
-    } catch (RuntimeException re) {
-      throw re;
-    } catch (Exception e) {
-      throw new RuntimeException("Failed to create iterator", e);
+    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+      throw new IllegalStateException("Failed to create iterator", e);
     }
   }
 
@@ -196,7 +193,7 @@
           if (fatalException instanceof RuntimeException) {
             throw (RuntimeException) fatalException;
           } else {
-            throw new RuntimeException(fatalException);
+            throw new IllegalStateException(fatalException);
           }
         }
 
@@ -207,13 +204,14 @@
               + " so that it can be closed when this Iterator is exhausted. Not"
               + " retaining a reference to the BatchScanner guarantees that you are"
               + " leaking threads in your client JVM.", shortMsg);
-          throw new RuntimeException(shortMsg + " Ensure proper handling of the BatchScanner.");
+          throw new IllegalStateException(
+              shortMsg + " Ensure proper handling of the BatchScanner.");
         }
 
         batchIterator = batch.iterator();
         return batch != LAST_BATCH;
       } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
   }
@@ -253,9 +251,9 @@
 
     int lastFailureSize = Integer.MAX_VALUE;
 
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(10, SECONDS).backOffFactor(1.07)
-        .logInterval(1, MINUTES).createFactory().createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(10)).backOffFactor(1.07)
+        .logInterval(Duration.ofMinutes(1)).createFactory().createRetry();
 
     while (true) {
 
@@ -286,7 +284,7 @@
         try {
           retry.waitForNextAttempt(log, "binRanges retry failures");
         } catch (InterruptedException e) {
-          throw new RuntimeException(e);
+          throw new IllegalStateException(e);
         }
 
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
index 980ba04..d54d144 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
-import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.function.Function.identity;
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toMap;
@@ -29,6 +27,7 @@
 import java.lang.management.CompilationMXBean;
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -61,6 +60,7 @@
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletServerMutations;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.constraints.Violations;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
@@ -71,14 +71,12 @@
 import org.apache.accumulo.core.dataImpl.TabletIdImpl;
 import org.apache.accumulo.core.dataImpl.thrift.TMutation;
 import org.apache.accumulo.core.dataImpl.thrift.UpdateErrors;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.core.lock.ServiceLock;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.tabletingest.thrift.TabletIngestClientService;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
@@ -90,6 +88,7 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
+import com.google.common.net.HostAndPort;
 
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
@@ -506,7 +505,7 @@
         wait();
       }
     } catch (InterruptedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
@@ -949,13 +948,13 @@
       // happen after the batch writer closes. See #3721
       try {
         final HostAndPort parsedServer = HostAndPort.fromString(location);
-        final TabletClientService.Iface client;
+        final TabletIngestClientService.Iface client;
 
         if (timeoutTracker.getTimeOut() < context.getClientTimeoutInMillis()) {
-          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context,
               timeoutTracker.getTimeOut());
         } else {
-          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+          client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context);
         }
 
         try {
@@ -1093,9 +1092,9 @@
 
       private void cancelSession() throws InterruptedException, ThriftSecurityException {
 
-        Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-            .incrementBy(100, MILLISECONDS).maxWait(60, SECONDS).backOffFactor(1.5)
-            .logInterval(3, MINUTES).createRetry();
+        Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+            .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(1)).backOffFactor(1.5)
+            .logInterval(Duration.ofMinutes(3)).createRetry();
 
         final HostAndPort parsedServer = HostAndPort.fromString(location);
 
@@ -1108,7 +1107,7 @@
         // exceptions.
         while (!somethingFailed.get()) {
 
-          TabletClientService.Client client = null;
+          TabletIngestClientService.Client client = null;
 
           // Check if a lock is held by any tserver at the host and port. It does not need to be the
           // exact tserver instance that existed when the session was created because if a new
@@ -1123,10 +1122,10 @@
 
           try {
             if (timeout < context.getClientTimeoutInMillis()) {
-              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context,
+              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context,
                   timeout);
             } else {
-              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_SERVER, parsedServer, context);
+              client = ThriftUtil.getClient(ThriftClientTypes.TABLET_INGEST, parsedServer, context);
             }
 
             if (useCloseUpdate) {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
index c76277f..1402732 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
@@ -19,9 +19,9 @@
 package org.apache.accumulo.core.clientImpl;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
 import java.io.IOException;
-import java.security.SecureRandom;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -34,7 +34,6 @@
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
@@ -46,6 +45,7 @@
 import org.apache.accumulo.core.client.TimedOutException;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.TabletLocator.TabletLocation;
+import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Column;
@@ -69,15 +69,13 @@
 import org.apache.accumulo.core.spi.scan.ScanServerAttempt;
 import org.apache.accumulo.core.spi.scan.ScanServerSelections;
 import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletscan.thrift.ScanServerBusyException;
+import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
-import org.apache.accumulo.core.tabletserver.thrift.ScanServerBusyException;
-import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
-import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.hadoop.io.Text;
@@ -86,6 +84,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.net.HostAndPort;
+
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Scope;
 
@@ -105,7 +105,6 @@
   // metadata
   public static final Map<TabletType,Set<String>> serversWaitedForWrites =
       new EnumMap<>(TabletType.class);
-  private static final SecureRandom random = new SecureRandom();
 
   static {
     for (TabletType ttype : TabletType.values()) {
@@ -270,9 +269,9 @@
   static <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration maxWaitTime,
       String description, Duration timeoutLeft, ClientContext context, TableId tableId,
       Logger log) {
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, TimeUnit.MILLISECONDS)
-        .incrementBy(100, TimeUnit.MILLISECONDS).maxWait(1, SECONDS).backOffFactor(1.5)
-        .logInterval(3, TimeUnit.MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofSeconds(1)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     long startTime = System.nanoTime();
     Optional<T> optional = condition.get();
@@ -319,7 +318,7 @@
       Thread.sleep(millis);
     }
     // wait 2 * last time, with +-10% random jitter
-    return (long) (Math.min(millis * 2, maxSleep) * (.9 + random.nextDouble() / 5));
+    return (long) (Math.min(millis * 2, maxSleep) * (.9 + RANDOM.get().nextDouble() / 5));
   }
 
   public static List<KeyValue> scan(ClientContext context, ScanState scanState, long timeOut)
@@ -374,18 +373,18 @@
             } else {
               // when a tablet splits we do want to continue scanning the low child
               // of the split if we are already passed it
-              Range dataRange = loc.tablet_extent.toDataRange();
+              Range dataRange = loc.getExtent().toDataRange();
 
               if (scanState.range.getStartKey() != null
                   && dataRange.afterEndKey(scanState.range.getStartKey())) {
                 // go to the next tablet
-                scanState.startRow = loc.tablet_extent.endRow();
+                scanState.startRow = loc.getExtent().endRow();
                 scanState.skipStartRow = true;
                 loc = null;
               } else if (scanState.range.getEndKey() != null
                   && dataRange.beforeStartKey(scanState.range.getEndKey())) {
                 // should not happen
-                throw new RuntimeException("Unexpected tablet, extent : " + loc.tablet_extent
+                throw new IllegalStateException("Unexpected tablet, extent : " + loc.getExtent()
                     + "  range : " + scanState.range + " startRow : " + scanState.startRow);
               }
             }
@@ -411,7 +410,7 @@
         }
 
         Span child2 = TraceUtil.startSpan(ThriftScanner.class, "scan::location",
-            Map.of("tserver", loc.tablet_location));
+            Map.of("tserver", loc.getTserverLocation()));
         try (Scope scanLocation = child2.makeCurrent()) {
           results = scan(loc, scanState, context, timeOut, startTime);
         } catch (AccumuloSecurityException e) {
@@ -422,7 +421,7 @@
           throw e;
         } catch (TApplicationException tae) {
           TraceUtil.setException(child2, tae, true);
-          throw new AccumuloServerException(scanState.getErrorLocation().tablet_location, tae);
+          throw new AccumuloServerException(scanState.getErrorLocation().getTserverLocation(), tae);
         } catch (TSampleNotPresentException tsnpe) {
           String message = "Table " + context.getPrintableTableInfoFromId(scanState.tableId)
               + " does not have sampling configured or built";
@@ -437,7 +436,7 @@
           }
           lastError = error;
 
-          TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.tablet_extent);
+          TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.getExtent());
           loc = null;
 
           // no need to try the current scan id somewhere else
@@ -512,7 +511,7 @@
           sleepMillis = pause(sleepMillis, maxSleepTime, scanState.runOnScanServer);
         } catch (TException e) {
           TabletLocator.getLocator(context, scanState.tableId).invalidateCache(context,
-              loc.tablet_location);
+              loc.getTserverLocation());
           error = "Scan failed, thrift error " + e.getClass().getName() + "  " + e.getMessage()
               + " " + scanState.getErrorLocation();
           if (!error.equals(lastError)) {
@@ -564,17 +563,17 @@
 
       TabletLocation newLoc;
 
-      var tabletId = new TabletIdImpl(loc.tablet_extent);
+      var tabletId = new TabletIdImpl(loc.getExtent());
 
       if (scanState.scanID != null && scanState.prevLoc != null
-          && scanState.prevLoc.tablet_session.equals("scan_server")
-          && scanState.prevLoc.tablet_extent.equals(loc.tablet_extent)) {
+          && scanState.prevLoc.getTserverSession().equals("scan_server")
+          && scanState.prevLoc.getExtent().equals(loc.getExtent())) {
         // this is the case of continuing a scan on a scan server for the same tablet, so lets not
         // call the scan server selector and just go back to the previous scan server
         newLoc = scanState.prevLoc;
         log.trace(
             "For tablet {} continuing scan on scan server {} without consulting scan server selector, using busyTimeout {}",
-            loc.tablet_extent, newLoc.tablet_location, scanState.busyTimeout);
+            loc.getExtent(), newLoc.getTserverLocation(), scanState.busyTimeout);
       } else {
         // obtain a snapshot once and only expose this snapshot to the plugin for consistency
         var attempts = scanState.scanAttempts.snapshot();
@@ -606,7 +605,7 @@
           public <T> Optional<T> waitUntil(Supplier<Optional<T>> condition, Duration maxWaitTime,
               String description) {
             return ThriftScanner.waitUntil(condition, maxWaitTime, description, timeoutLeft,
-                context, loc.tablet_extent.tableId(), log);
+                context, loc.getExtent().tableId(), log);
           }
         };
 
@@ -616,19 +615,18 @@
 
         String scanServer = actions.getScanServer(tabletId);
         if (scanServer != null) {
-          newLoc = new TabletLocation(loc.tablet_extent, scanServer, "scan_server");
+          newLoc = new TabletLocation(loc.getExtent(), scanServer, "scan_server");
           delay = actions.getDelay();
           scanState.busyTimeout = actions.getBusyTimeout();
           log.trace(
               "For tablet {} using hints {} scan server selector chose scan_server:{} delay:{} busyTimeout:{}",
-              loc.tablet_extent, scanState.executionHints, scanServer, delay,
-              scanState.busyTimeout);
+              loc.getExtent(), scanState.executionHints, scanServer, delay, scanState.busyTimeout);
         } else {
           newLoc = loc;
           delay = actions.getDelay();
           scanState.busyTimeout = Duration.ZERO;
           log.trace("For tablet {} using hints {} scan server selector chose tablet_server",
-              loc.tablet_extent, scanState.executionHints);
+              loc.getExtent(), scanState.executionHints);
         }
 
         if (!delay.isZero()) {
@@ -641,7 +639,7 @@
         }
       }
 
-      var reporter = scanState.scanAttempts.createReporter(newLoc.tablet_location, tabletId);
+      var reporter = scanState.scanAttempts.createReporter(newLoc.getTserverLocation(), tabletId);
 
       try {
         return scanRpc(newLoc, scanState, context, scanState.busyTimeout.toMillis());
@@ -666,7 +664,7 @@
 
     final TInfo tinfo = TraceUtil.traceInfo();
 
-    final HostAndPort parsedLocation = HostAndPort.fromString(loc.tablet_location);
+    final HostAndPort parsedLocation = HostAndPort.fromString(loc.getTserverLocation());
     TabletScanClientService.Client client =
         ThriftUtil.getClient(ThriftClientTypes.TABLET_SCAN, parsedLocation, context);
 
@@ -681,23 +679,24 @@
       scanState.prevLoc = loc;
 
       if (scanState.scanID == null) {
-        Thread.currentThread().setName("Starting scan tserver=" + loc.tablet_location + " tableId="
-            + loc.tablet_extent.tableId());
+        Thread.currentThread().setName("Starting scan tserver=" + loc.getTserverLocation()
+            + " tableId=" + loc.getExtent().tableId());
 
         if (log.isTraceEnabled()) {
-          String msg = "Starting scan tserver=" + loc.tablet_location + " tablet="
-              + loc.tablet_extent + " range=" + scanState.range + " ssil="
+          String msg = "Starting scan tserver=" + loc.getTserverLocation() + " tablet="
+              + loc.getExtent() + " range=" + scanState.range + " ssil="
               + scanState.serverSideIteratorList + " ssio=" + scanState.serverSideIteratorOptions
               + " context=" + scanState.classLoaderContext;
           log.trace("tid={} {}", Thread.currentThread().getId(), msg);
           timer = new OpTimer().start();
         }
 
-        TabletType ttype = TabletType.type(loc.tablet_extent);
-        boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(loc.tablet_location);
+        TabletType ttype = TabletType.type(loc.getExtent());
+        boolean waitForWrites =
+            !serversWaitedForWrites.get(ttype).contains(loc.getTserverLocation());
 
         InitialScan is = client.startScan(tinfo, scanState.context.rpcCreds(),
-            loc.tablet_extent.toThrift(), scanState.range.toThrift(),
+            loc.getExtent().toThrift(), scanState.range.toThrift(),
             scanState.columns.stream().map(Column::toThrift).collect(Collectors.toList()),
             scanState.size, scanState.serverSideIteratorList, scanState.serverSideIteratorOptions,
             scanState.authorizations.getAuthorizationsBB(), waitForWrites, scanState.isolated,
@@ -705,7 +704,7 @@
             SamplerConfigurationImpl.toThrift(scanState.samplerConfig), scanState.batchTimeOut,
             scanState.classLoaderContext, scanState.executionHints, busyTimeout);
         if (waitForWrites) {
-          serversWaitedForWrites.get(ttype).add(loc.tablet_location);
+          serversWaitedForWrites.get(ttype).add(loc.getTserverLocation());
         }
 
         sr = is.result;
@@ -719,7 +718,7 @@
       } else {
         // log.debug("Calling continue scan : "+scanState.range+" loc = "+loc);
         String msg =
-            "Continuing scan tserver=" + loc.tablet_location + " scanid=" + scanState.scanID;
+            "Continuing scan tserver=" + loc.getTserverLocation() + " scanid=" + scanState.scanID;
         Thread.currentThread().setName(msg);
 
         if (log.isTraceEnabled()) {
@@ -744,7 +743,7 @@
       } else {
         // log.debug("No more : tab end row = "+loc.tablet_extent.getEndRow()+" range =
         // "+scanState.range);
-        if (loc.tablet_extent.endRow() == null) {
+        if (loc.getExtent().endRow() == null) {
           scanState.finished = true;
 
           if (timer != null) {
@@ -755,8 +754,8 @@
           }
 
         } else if (scanState.range.getEndKey() == null || !scanState.range
-            .afterEndKey(new Key(loc.tablet_extent.endRow()).followingKey(PartialKey.ROW))) {
-          scanState.startRow = loc.tablet_extent.endRow();
+            .afterEndKey(new Key(loc.getExtent().endRow()).followingKey(PartialKey.ROW))) {
+          scanState.startRow = loc.getExtent().endRow();
           scanState.skipStartRow = true;
 
           if (timer != null) {
@@ -803,7 +802,7 @@
       TInfo tinfo = TraceUtil.traceInfo();
 
       log.debug("Closing active scan {} {}", scanState.prevLoc, scanState.scanID);
-      HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.tablet_location);
+      HostAndPort parsedLocation = HostAndPort.fromString(scanState.prevLoc.getTserverLocation());
       TabletScanClientService.Client client = null;
       try {
         client =
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
index f4c7047..0f84154 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportKey.java
@@ -25,9 +25,9 @@
 import org.apache.accumulo.core.rpc.SaslConnectionParams;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.net.HostAndPort;
 
 @VisibleForTesting
 public class ThriftTransportKey {
@@ -58,7 +58,7 @@
     this.saslParams = saslParams;
     if (saslParams != null && sslParams != null) {
       // TSasl and TSSL transport factories don't play nicely together
-      throw new RuntimeException("Cannot use both SSL and SASL thrift transports");
+      throw new IllegalArgumentException("Cannot use both SSL and SASL thrift transports");
     }
     this.hash = Objects.hash(type, server, timeout, sslParams, saslParams);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
index 6f62125..1863cf6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftTransportPool.java
@@ -20,8 +20,8 @@
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.accumulo.core.util.LazySingletons.RANDOM;
 
-import java.security.SecureRandom;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -43,7 +43,6 @@
 
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
-import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.thrift.TConfiguration;
@@ -53,13 +52,13 @@
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
+import com.google.common.net.HostAndPort;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 public class ThriftTransportPool {
 
   private static final Logger log = LoggerFactory.getLogger(ThriftTransportPool.class);
-  private static final SecureRandom random = new SecureRandom();
   private static final long ERROR_THRESHOLD = 20L;
   private static final long STUCK_THRESHOLD = MINUTES.toMillis(2);
 
@@ -134,7 +133,7 @@
     if (serversSet.isEmpty()) {
       return null;
     }
-    Collections.shuffle(serversSet, random);
+    Collections.shuffle(serversSet, RANDOM.get());
     for (ThriftTransportKey ttk : serversSet) {
       CachedConnection connection = connectionPool.reserveAny(ttk);
       if (connection != null) {
@@ -244,7 +243,7 @@
     try {
       checkThread.join();
     } catch (InterruptedException e) {
-      throw new RuntimeException(e);
+      throw new IllegalStateException(e);
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
index 2c71bc5..e64ae66 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/UserCompactionUtils.java
@@ -217,8 +217,6 @@
         is.write(dout);
       }
 
-      CompactionStrategyConfigUtil.encode(dout, cc);
-
       encodeConfigurer(dout, cc.getConfigurer());
       encodeSelector(dout, cc.getSelector());
       encode(dout, cc.getExecutionHints());
@@ -260,8 +258,6 @@
 
       cc.setIterators(iterators);
 
-      CompactionStrategyConfigUtil.decode(cc, din);
-
       var configurer = decodeConfigurer(din);
       if (!isDefault(configurer)) {
         cc.setConfigurer(configurer);
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
deleted file mode 100644
index 9f124ed..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/Writer.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.metadata.schema.Ample;
-
-public class Writer {
-
-  private ClientContext context;
-  private TableId tableId;
-
-  public Writer(ClientContext context, TableId tableId) {
-    checkArgument(context != null, "context is null");
-    checkArgument(tableId != null, "tableId is null");
-    this.context = context;
-    this.tableId = tableId;
-  }
-
-  public void update(Mutation m) throws AccumuloException, TableNotFoundException {
-    checkArgument(m != null, "m is null");
-
-    if (m.size() == 0) {
-      throw new IllegalArgumentException("Can not add empty mutations");
-    }
-
-    String table = Ample.DataLevel.of(tableId).metaTable();
-
-    try (var writer = context.createBatchWriter(table)) {
-      writer.addMutation(m);
-    }
-
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
index 23f036b..39c7cec 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ZookeeperLockChecker.java
@@ -20,8 +20,8 @@
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.clientImpl.TabletLocatorImpl.TabletServerLockChecker;
-import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.lock.ServiceLock;
 
 public class ZookeeperLockChecker implements TabletServerLockChecker {
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
index f13420d..8689a56 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java
@@ -19,8 +19,6 @@
 package org.apache.accumulo.core.clientImpl.bulk;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static java.util.concurrent.TimeUnit.MINUTES;
 import static java.util.stream.Collectors.groupingBy;
 import static org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile.pathToCacheId;
 import static org.apache.accumulo.core.util.Validators.EXISTING_TABLE_NAME;
@@ -28,6 +26,7 @@
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -74,6 +73,7 @@
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.metadata.UnreferencedTabletFile;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
@@ -85,9 +85,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
 import com.google.common.base.Preconditions;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
 import com.google.common.collect.Sets;
 
 public class BulkImport implements ImportDestinationArguments, ImportMappingOptions {
@@ -141,9 +141,9 @@
     if (propValue != null) {
       maxTablets = Integer.parseInt(propValue);
     }
-    Retry retry = Retry.builder().infiniteRetries().retryAfter(100, MILLISECONDS)
-        .incrementBy(100, MILLISECONDS).maxWait(2, MINUTES).backOffFactor(1.5)
-        .logInterval(3, MINUTES).createRetry();
+    Retry retry = Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(100))
+        .incrementBy(Duration.ofMillis(100)).maxWait(Duration.ofMinutes(2)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
 
     // retry if a merge occurs
     boolean shouldRetry = true;
@@ -179,7 +179,7 @@
         try {
           retry.waitForNextAttempt(log, String.format("bulk import to %s(%s)", tableName, tableId));
         } catch (InterruptedException e) {
-          throw new RuntimeException(e);
+          throw new IllegalStateException(e);
         }
         log.info(ae.getMessage() + ". Retrying bulk import to " + tableName);
       }
@@ -261,9 +261,9 @@
     long l;
   }
 
-  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile,
-      long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
-      CryptoService cs) throws IOException {
+  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf,
+      UnreferencedTabletFile dataFile, long fileSize, Collection<KeyExtent> extents, FileSystem ns,
+      Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {
 
     if (extents.size() == 1) {
       return Collections.singletonMap(extents.iterator().next(), fileSize);
@@ -277,17 +277,14 @@
 
     Text row = new Text();
 
-    FileSKVIterator index = FileOperations.getInstance().newIndexReaderBuilder()
-        .forFile(mapFile.toString(), ns, ns.getConf(), cs).withTableConfiguration(acuConf)
-        .withFileLenCache(fileLenCache).build();
-
-    try {
+    try (FileSKVIterator index =
+        FileOperations.getInstance().newIndexReaderBuilder().forFile(dataFile, ns, ns.getConf(), cs)
+            .withTableConfiguration(acuConf).withFileLenCache(fileLenCache).build()) {
       while (index.hasTop()) {
         Key key = index.getTopKey();
         totalIndexEntries++;
         key.getRow(row);
 
-        // TODO this could use a binary search
         for (Entry<KeyExtent,MLong> entry : counts.entrySet()) {
           if (entry.getKey().contains(row)) {
             entry.getValue().l++;
@@ -296,14 +293,6 @@
 
         index.next();
       }
-    } finally {
-      try {
-        if (index != null) {
-          index.close();
-        }
-      } catch (IOException e) {
-        log.debug("Failed to close " + mapFile, e);
-      }
     }
 
     Map<KeyExtent,Long> results = new TreeMap<>();
@@ -354,12 +343,11 @@
   }
 
   public static List<KeyExtent> findOverlappingTablets(ClientContext context,
-      KeyExtentCache extentCache, Path file, FileSystem fs, Cache<String,Long> fileLenCache,
-      CryptoService cs) throws IOException {
+      KeyExtentCache extentCache, UnreferencedTabletFile file, FileSystem fs,
+      Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {
     try (FileSKVIterator reader = FileOperations.getInstance().newReaderBuilder()
-        .forFile(file.toString(), fs, fs.getConf(), cs)
-        .withTableConfiguration(context.getConfiguration()).withFileLenCache(fileLenCache)
-        .seekToBeginning().build()) {
+        .forFile(file, fs, fs.getConf(), cs).withTableConfiguration(context.getConfiguration())
+        .withFileLenCache(fileLenCache).seekToBeginning().build()) {
       return findOverlappingTablets(extentCache, reader);
     }
   }
@@ -379,7 +367,7 @@
     Map<String,Long> absFileLens = new HashMap<>();
     fileLens.forEach((k, v) -> absFileLens.put(pathToCacheId(new Path(dir, k)), v));
 
-    Cache<String,Long> fileLenCache = CacheBuilder.newBuilder().build();
+    Cache<String,Long> fileLenCache = Caffeine.newBuilder().build();
 
     fileLenCache.putAll(absFileLens);
 
@@ -411,13 +399,7 @@
     fileDestinations.values().stream().flatMap(List::stream)
         .filter(dest -> dest.getRangeType() == RangeType.FILE)
         .flatMap(dest -> Stream.of(dest.getStartRow(), dest.getEndRow())).filter(Objects::nonNull)
-        .map(Text::new).sorted().distinct().forEach(row -> {
-          try {
-            extentCache.lookup(row);
-          } catch (Exception e) {
-            throw new RuntimeException(e);
-          }
-        });
+        .map(Text::new).sorted().distinct().forEach(extentCache::lookup);
 
     SortedMap<KeyExtent,Files> mapping = new TreeMap<>();
 
@@ -548,22 +530,22 @@
         context.instanceOperations().getSystemConfiguration(), tableProps, tableId);
 
     for (FileStatus fileStatus : files) {
-      Path filePath = fileStatus.getPath();
+      UnreferencedTabletFile file = UnreferencedTabletFile.of(fs, fileStatus.getPath());
       CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
         try {
           long t1 = System.currentTimeMillis();
           List<KeyExtent> extents =
-              findOverlappingTablets(context, extentCache, filePath, fs, fileLensCache, cs);
+              findOverlappingTablets(context, extentCache, file, fs, fileLensCache, cs);
           // make sure file isn't going to too many tablets
-          checkTabletCount(maxTablets, extents.size(), filePath.toString());
-          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), filePath,
+          checkTabletCount(maxTablets, extents.size(), file.toString());
+          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), file,
               fileStatus.getLen(), extents, fs, fileLensCache, cs);
           Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
           for (KeyExtent ke : extents) {
-            pathLocations.put(ke, new Bulk.FileInfo(filePath, estSizes.getOrDefault(ke, 0L)));
+            pathLocations.put(ke, new Bulk.FileInfo(file.getPath(), estSizes.getOrDefault(ke, 0L)));
           }
           long t2 = System.currentTimeMillis();
-          log.debug("Mapped {} to {} tablets in {}ms", filePath, pathLocations.size(), t2 - t1);
+          log.debug("Mapped {} to {} tablets in {}ms", file, pathLocations.size(), t2 - t1);
           return pathLocations;
         } catch (Exception e) {
           throw new CompletionException(e);
@@ -581,9 +563,9 @@
         pathMapping.forEach((ext, fi) -> mappings.computeIfAbsent(ext, k -> new Files()).add(fi));
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       } catch (ExecutionException e) {
-        throw new RuntimeException(e);
+        throw new IllegalStateException(e);
       }
     }
 
@@ -606,7 +588,7 @@
         if (ke.contains(oke)) {
           mappings.get(ke).merge(mappings.remove(oke));
         } else if (!oke.contains(ke)) {
-          throw new RuntimeException("Error during bulk import: Unable to merge overlapping "
+          throw new IllegalStateException("Error during bulk import: Unable to merge overlapping "
               + "tablets where neither tablet contains the other. This may be caused by "
               + "a concurrent merge. Key extents " + oke + " and " + ke + " overlap, but "
               + "neither contains the other.");
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
index 0f8d46c..533a43c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkSerialize.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.core.clientImpl.bulk;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.util.LazySingletons.GSON;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
@@ -27,10 +28,6 @@
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
-import java.lang.reflect.Type;
-import java.util.Base64;
-import java.util.Base64.Decoder;
-import java.util.Base64.Encoder;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -41,17 +38,10 @@
 import org.apache.accumulo.core.clientImpl.bulk.Bulk.Mapping;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
 import org.apache.hadoop.fs.Path;
 
 import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonParseException;
-import com.google.gson.JsonPrimitive;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
 import com.google.gson.reflect.TypeToken;
 import com.google.gson.stream.JsonWriter;
 
@@ -60,28 +50,7 @@
  */
 public class BulkSerialize {
 
-  private static class ByteArrayToBase64TypeAdapter
-      implements JsonSerializer<byte[]>, JsonDeserializer<byte[]> {
-
-    Decoder decoder = Base64.getUrlDecoder();
-    Encoder encoder = Base64.getUrlEncoder();
-
-    @Override
-    public byte[] deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
-        throws JsonParseException {
-      return decoder.decode(json.getAsString());
-    }
-
-    @Override
-    public JsonElement serialize(byte[] src, Type typeOfSrc, JsonSerializationContext context) {
-      return new JsonPrimitive(encoder.encodeToString(src));
-    }
-  }
-
-  static Gson createGson() {
-    return new GsonBuilder()
-        .registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter()).create();
-  }
+  private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
 
   public interface Output {
     OutputStream create(Path path) throws IOException;
@@ -100,7 +69,6 @@
 
     try (OutputStream fsOut = output.create(lmFile); JsonWriter writer =
         new JsonWriter(new BufferedWriter(new OutputStreamWriter(fsOut, UTF_8)))) {
-      Gson gson = createGson();
       writer.setIndent("  ");
       writer.beginArray();
       Set<Entry<KeyExtent,Files>> es = loadMapping.entrySet();
@@ -130,7 +98,7 @@
     final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
     try (OutputStream fsOut = output.create(renamingFile);
         BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fsOut))) {
-      new Gson().toJson(oldToNewNameMap, writer);
+      GSON.get().toJson(oldToNewNameMap, writer);
     }
   }
 
@@ -141,7 +109,6 @@
   public static Map<String,String> readRenameMap(String bulkDir, Input input) throws IOException {
     final Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
     Map<String,String> oldToNewNameMap;
-    Gson gson = createGson();
     try (InputStream fis = input.open(renamingFile);
         BufferedReader reader = new BufferedReader(new InputStreamReader(fis))) {
       oldToNewNameMap = gson.fromJson(reader, new TypeToken<Map<String,String>>() {}.getType());
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
index f17639a..e338fe9 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/ConcurrentKeyExtentCache.java
@@ -23,7 +23,6 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Objects;
@@ -88,9 +87,9 @@
 
   @VisibleForTesting
   protected Stream<KeyExtent> lookupExtents(Text row) {
-    return TabletsMetadata.builder(ctx).forTable(tableId).overlapping(row, true, null)
-        .checkConsistency().fetch(PREV_ROW).build().stream().limit(100)
-        .map(TabletMetadata::getExtent);
+    TabletsMetadata tabletsMetadata = TabletsMetadata.builder(ctx).forTable(tableId)
+        .overlapping(row, true, null).checkConsistency().fetch(PREV_ROW).build();
+    return tabletsMetadata.stream().limit(100).map(TabletMetadata::getExtent);
   }
 
   @Override
@@ -129,15 +128,8 @@
         for (Text lookupRow : lookupRows) {
           if (getFromCache(lookupRow) == null) {
             while (true) {
-              try {
-                Iterator<KeyExtent> iter = lookupExtents(lookupRow).iterator();
-                while (iter.hasNext()) {
-                  KeyExtent ke2 = iter.next();
-                  if (inCache(ke2)) {
-                    break;
-                  }
-                  updateCache(ke2);
-                }
+              try (Stream<KeyExtent> keyExtentStream = lookupExtents(lookupRow)) {
+                keyExtentStream.takeWhile(ke2 -> !inCache(ke2)).forEach(this::updateCache);
                 break;
               } catch (TabletDeletedException tde) {
                 // tablets were merged away in the table, start over and try again
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
index 30e1041..36b2d5b 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/LoadMappingIterator.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.clientImpl.bulk;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.clientImpl.bulk.BulkSerialize.createGson;
 
 import java.io.BufferedReader;
 import java.io.IOException;
@@ -32,6 +31,7 @@
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.json.ByteArrayToBase64TypeAdapter;
 
 import com.google.gson.Gson;
 import com.google.gson.stream.JsonReader;
@@ -41,14 +41,14 @@
  */
 public class LoadMappingIterator
     implements Iterator<Map.Entry<KeyExtent,Bulk.Files>>, AutoCloseable {
-  private TableId tableId;
-  private JsonReader reader;
-  private Gson gson = createGson();
+  private final TableId tableId;
+  private final JsonReader reader;
+  private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
   private Map<String,String> renameMap;
 
-  LoadMappingIterator(TableId tableId, InputStream loadMapFile) throws IOException {
+  LoadMappingIterator(TableId tableId, InputStream loadMappingFile) throws IOException {
     this.tableId = tableId;
-    this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMapFile, UTF_8)));
+    this.reader = new JsonReader(new BufferedReader(new InputStreamReader(loadMappingFile, UTF_8)));
     this.reader.beginArray();
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
deleted file mode 100644
index db69081..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapred/BatchInputSplit.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapred;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.hadoop.mapred.InputSplit;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * The Class BatchInputSplit. Encapsulates Accumulo ranges for use in Map Reduce jobs. Can contain
- * several Ranges per InputSplit.
- *
- * @deprecated since 2.0.0
- */
-@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
-    justification = "Intended to share code between mapred and mapreduce")
-@Deprecated(since = "2.0.0")
-public class BatchInputSplit extends org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit
-    implements InputSplit {
-
-  public BatchInputSplit() {}
-
-  public BatchInputSplit(BatchInputSplit split) throws IOException {
-    super(split);
-  }
-
-  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
-      String[] location) {
-    super(table, tableId, ranges, location);
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
deleted file mode 100644
index cc01471..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplit.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.PartialKey;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-
-/**
- * The Class BatchInputSplit. Encapsulates a set of Accumulo ranges on a single tablet for use in
- * Map Reduce jobs. Can contain several Ranges per split.
- *
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class BatchInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit {
-  private Collection<Range> ranges;
-  private float[] rangeProgress = null;
-
-  public BatchInputSplit() {
-    ranges = Collections.emptyList();
-  }
-
-  public BatchInputSplit(BatchInputSplit split) throws IOException {
-    super(split);
-    this.setRanges(split.getRanges());
-  }
-
-  public BatchInputSplit(String table, TableId tableId, Collection<Range> ranges,
-      String[] locations) {
-    super(table, tableId.canonical(), new Range(), locations);
-    this.ranges = ranges;
-  }
-
-  /**
-   * Save progress on each call to this function, implied by value of currentKey, and return average
-   * ranges in the split
-   */
-  @Override
-  public float getProgress(Key currentKey) {
-    if (rangeProgress == null) {
-      rangeProgress = new float[ranges.size()];
-    }
-
-    float total = 0; // progress per range could be on different scales, this number is "fuzzy"
-
-    if (currentKey == null) {
-      for (float progress : rangeProgress) {
-        total += progress;
-      }
-    } else {
-      int i = 0;
-      for (Range range : ranges) {
-        if (range.contains(currentKey)) {
-          // find the current range and report as if that is the single range
-          if (range.getStartKey() != null && range.getEndKey() != null) {
-            if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
-              // just look at the row progress
-              rangeProgress[i] = SplitUtils.getProgress(range.getStartKey().getRowData(),
-                  range.getEndKey().getRowData(), currentKey.getRowData());
-            } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM)
-                != 0) {
-              // just look at the column family progress
-              rangeProgress[i] = SplitUtils.getProgress(range.getStartKey().getColumnFamilyData(),
-                  range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-            } else if (range.getStartKey().compareTo(range.getEndKey(),
-                PartialKey.ROW_COLFAM_COLQUAL) != 0) {
-              // just look at the column qualifier progress
-              rangeProgress[i] = SplitUtils.getProgress(
-                  range.getStartKey().getColumnQualifierData(),
-                  range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
-            }
-          }
-          total += rangeProgress[i];
-        }
-        i++;
-      }
-    }
-
-    return total / ranges.size();
-  }
-
-  /**
-   * This implementation of length is only an estimate, it does not provide exact values. Do not
-   * have your code rely on this return value.
-   */
-  @Override
-  public long getLength() {
-    long sum = 0;
-    for (Range range : ranges) {
-      sum += SplitUtils.getRangeLength(range);
-    }
-    return sum;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-
-    int numRanges = in.readInt();
-    ranges = new ArrayList<>(numRanges);
-    for (int i = 0; i < numRanges; ++i) {
-      Range r = new Range();
-      r.readFields(in);
-      ranges.add(r);
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-
-    out.writeInt(ranges.size());
-    for (Range r : ranges) {
-      r.write(out);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(256);
-    sb.append("BatchInputSplit:");
-    sb.append(" Ranges: ").append(Arrays.asList(ranges));
-    sb.append(super.toString());
-    return sb.toString();
-  }
-
-  public void setRanges(Collection<Range> ranges) {
-    this.ranges = ranges;
-  }
-
-  public Collection<Range> getRanges() {
-    return ranges;
-  }
-
-  @Override
-  public Range getRange() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setRange(Range range) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Boolean isIsolatedScan() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setIsolatedScan(Boolean isolatedScan) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Boolean isOffline() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setOffline(Boolean offline) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Boolean usesLocalIterators() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setUsesLocalIterators(Boolean localIterators) {
-    throw new UnsupportedOperationException();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java
deleted file mode 100644
index 7871624..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/DelegationTokenStub.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-
-/**
- * An internal stub class for passing DelegationToken information out of the Configuration back up
- * to the appropriate implementation for mapreduce or mapred.
- *
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class DelegationTokenStub implements AuthenticationToken {
-
-  private String serviceName;
-
-  public DelegationTokenStub(String serviceName) {
-    requireNonNull(serviceName);
-    this.serviceName = serviceName;
-  }
-
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  @Override
-  public void write(DataOutput out) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void readFields(DataInput in) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void destroy() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean isDestroyed() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void init(Properties properties) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Set<TokenProperty> getProperties() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public AuthenticationToken clone() {
-    throw new UnsupportedOperationException();
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java
deleted file mode 100644
index 9634659..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/SplitUtils.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce;
-
-import java.math.BigInteger;
-
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Range;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-
-/**
- * @deprecated since 2.0.0
- */
-@Deprecated(since = "2.0.0")
-public class SplitUtils {
-
-  /**
-   * Central place to set common split configuration not handled by split constructors. The
-   * intention is to make it harder to miss optional setters in future refactor.
-   */
-  public static void updateSplit(org.apache.accumulo.core.client.mapreduce.RangeInputSplit split,
-      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig, Level logLevel) {
-    split.setFetchedColumns(tableConfig.getFetchedColumns());
-    split.setIterators(tableConfig.getIterators());
-    split.setLogLevel(logLevel);
-    split.setSamplerConfiguration(tableConfig.getSamplerConfiguration());
-  }
-
-  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-    BigInteger startBI = new BigInteger(SplitUtils.extractBytes(start, maxDepth));
-    BigInteger endBI = new BigInteger(SplitUtils.extractBytes(end, maxDepth));
-    BigInteger positionBI = new BigInteger(SplitUtils.extractBytes(position, maxDepth));
-    return (float) (positionBI.subtract(startBI).doubleValue()
-        / endBI.subtract(startBI).doubleValue());
-  }
-
-  public static long getRangeLength(Range range) {
-    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE})
-        : range.getStartKey().getRow();
-    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE})
-        : range.getEndKey().getRow();
-    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-    long diff = 0;
-
-    byte[] start = startRow.getBytes();
-    byte[] stop = stopRow.getBytes();
-    for (int i = 0; i < maxCommon; ++i) {
-      diff |= 0xff & (start[i] ^ stop[i]);
-      diff <<= Byte.SIZE;
-    }
-
-    if (startRow.getLength() != stopRow.getLength()) {
-      diff |= 0xff;
-    }
-
-    return diff + 1;
-  }
-
-  static byte[] extractBytes(ByteSequence seq, int numBytes) {
-    byte[] bytes = new byte[numBytes + 1];
-    bytes[0] = 0;
-    for (int i = 0; i < numBytes; i++) {
-      if (i >= seq.length()) {
-        bytes[i + 1] = 0;
-      } else {
-        bytes[i + 1] = seq.byteAt(i);
-      }
-    }
-    return bytes;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
deleted file mode 100644
index 6d4b3c6..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBase.java
+++ /dev/null
@@ -1,520 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static java.util.Objects.requireNonNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Base64;
-import java.util.Scanner;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.clientImpl.Credentials;
-import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
-@SuppressWarnings("deprecation")
-public class ConfiguratorBase {
-
-  protected static final Logger log = Logger.getLogger(ConfiguratorBase.class);
-
-  /**
-   * Specifies that connection info was configured
-   *
-   * @since 1.6.0
-   */
-  public enum ConnectorInfo {
-    IS_CONFIGURED, PRINCIPAL, TOKEN
-  }
-
-  public enum TokenSource {
-    FILE, INLINE, JOB;
-
-    private String prefix;
-
-    private TokenSource() {
-      prefix = name().toLowerCase() + ":";
-    }
-
-    public String prefix() {
-      return prefix;
-    }
-  }
-
-  /**
-   * Configuration keys for available Instance types.
-   *
-   * @since 1.6.0
-   */
-  public enum InstanceOpts {
-    TYPE, NAME, ZOO_KEEPERS, CLIENT_CONFIG
-  }
-
-  /**
-   * Configuration keys for general configuration options.
-   *
-   * @since 1.6.0
-   */
-  public enum GeneralOpts {
-    LOG_LEVEL, VISIBILITY_CACHE_SIZE
-  }
-
-  /**
-   * Provides a configuration key for a given feature enum, prefixed by the implementingClass
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param e the enum used to provide the unique part of the configuration key
-   * @return the configuration key
-   * @since 1.6.0
-   */
-  protected static String enumToConfKey(Class<?> implementingClass, Enum<?> e) {
-    return implementingClass.getSimpleName() + "." + e.getDeclaringClass().getSimpleName() + "."
-        + StringUtils.camelize(e.name().toLowerCase());
-  }
-
-  /**
-   * Provides a configuration key for a given feature enum.
-   *
-   * @param e the enum used to provide the unique part of the configuration key
-   * @return the configuration key
-   */
-  protected static String enumToConfKey(Enum<?> e) {
-    return e.getDeclaringClass().getSimpleName() + "."
-        + StringUtils.camelize(e.name().toLowerCase());
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all
-   * MapReduce tasks. It is BASE64 encoded to provide a charset safe conversion to a string, and is
-   * not intended to be secure.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param principal a valid Accumulo user name
-   * @param token the user's password
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf,
-      String principal, AuthenticationToken token) {
-    if (isConnectorInfoSet(implementingClass, conf)) {
-      throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName()
-          + " can only be set once per job");
-    }
-    checkArgument(principal != null, "principal is null");
-    checkArgument(token != null, "token is null");
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
-    if (token instanceof DelegationTokenImpl) {
-      // Avoid serializing the DelegationToken secret in the configuration -- the Job will do that
-      // work for us securely
-      DelegationTokenImpl delToken = (DelegationTokenImpl) token;
-      conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), TokenSource.JOB.prefix()
-          + token.getClass().getName() + ":" + delToken.getServiceName().toString());
-    } else {
-      conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
-          TokenSource.INLINE.prefix() + token.getClass().getName() + ":"
-              + Base64.getEncoder().encodeToString(AuthenticationTokenSerializer.serialize(token)));
-    }
-  }
-
-  private static String cachedTokenFileName(Class<?> implementingClass) {
-    return implementingClass.getSimpleName() + ".tokenfile";
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * Pulls a token file into the Distributed Cache that contains the authentication token in an
-   * attempt to be more secure than storing the password in the Configuration. Token file created
-   * with "bin/accumulo create-token".
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param principal a valid Accumulo user name
-   * @param tokenFile the path to the token file in DFS
-   * @since 1.6.0
-   */
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf,
-      String principal, String tokenFile) {
-    if (isConnectorInfoSet(implementingClass, conf)) {
-      throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName()
-          + " can only be set once per job");
-    }
-
-    checkArgument(principal != null, "principal is null");
-    checkArgument(tokenFile != null, "tokenFile is null");
-
-    DistributedCacheHelper.addCacheFile(tokenFile, cachedTokenFileName(implementingClass), conf);
-
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
-        TokenSource.FILE.prefix() + tokenFile);
-  }
-
-  /**
-   * Determines if the connector info has already been set for this instance.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return true if the connector info has already been set, false otherwise
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
-    return conf.getBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), false);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the principal
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
-    return conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
-  }
-
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the
-   * configuration, whichever was used when the job was configured.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   * @see #setConnectorInfo(Class, Configuration, String, String)
-   */
-  public static AuthenticationToken getAuthenticationToken(Class<?> implementingClass,
-      Configuration conf) {
-    String token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
-    if (token == null || token.isEmpty()) {
-      return null;
-    }
-    if (token.startsWith(TokenSource.INLINE.prefix())) {
-      String[] args = token.substring(TokenSource.INLINE.prefix().length()).split(":", 2);
-      if (args.length == 2) {
-        return AuthenticationTokenSerializer.deserialize(args[0],
-            Base64.getDecoder().decode(args[1]));
-      }
-    } else if (token.startsWith(TokenSource.FILE.prefix())) {
-      String tokenFileName = token.substring(TokenSource.FILE.prefix().length());
-      return getTokenFromFile(implementingClass, conf, getPrincipal(implementingClass, conf),
-          tokenFileName);
-    } else if (token.startsWith(TokenSource.JOB.prefix())) {
-      String[] args = token.substring(TokenSource.JOB.prefix().length()).split(":", 2);
-      if (args.length == 2) {
-        String className = args[0], serviceName = args[1];
-        if (DelegationTokenImpl.class.getName().equals(className)) {
-          return new org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub(serviceName);
-        }
-      }
-    }
-
-    throw new IllegalStateException("Token was not properly serialized into the configuration");
-  }
-
-  /**
-   * Reads from the token file in distributed cache. Currently, the token file stores data separated
-   * by colons e.g. principal:token_class:token
-   *
-   * @param conf the Hadoop context for the configured job
-   * @return path to the token file as a String
-   * @since 1.6.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  public static AuthenticationToken getTokenFromFile(Class<?> implementingClass, Configuration conf,
-      String principal, String tokenFile) {
-
-    try (InputStream inputStream = DistributedCacheHelper.openCachedFile(tokenFile,
-        cachedTokenFileName(implementingClass), conf)) {
-
-      try (Scanner fileScanner = new Scanner(inputStream, UTF_8)) {
-        while (fileScanner.hasNextLine()) {
-          Credentials creds = Credentials.deserialize(fileScanner.nextLine());
-          if (principal.equals(creds.getPrincipal())) {
-            return creds.getToken();
-          }
-        }
-        throw new IllegalArgumentException("No token found for " + principal);
-      }
-
-    } catch (IOException e) {
-      throw new IllegalStateException("Error closing token file stream", e);
-    }
-
-  }
-
-  /**
-   * Configures a ZooKeeperInstance for this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param clientConfig client configuration for specifying connection timeouts, SSL connection
-   *        options, etc.
-   * @since 1.6.0
-   */
-  public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf,
-      org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    String key = enumToConfKey(implementingClass, InstanceOpts.TYPE);
-    if (!conf.get(key, "").isEmpty()) {
-      throw new IllegalStateException(
-          "Instance info can only be set once per job; it has already been configured with "
-              + conf.get(key));
-    }
-    conf.set(key, "ZooKeeperInstance");
-    if (clientConfig != null) {
-      conf.set(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG),
-          clientConfig.serialize());
-    }
-  }
-
-  /**
-   * Initializes an Accumulo Instance based on the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return an Accumulo instance
-   * @since 1.6.0
-   */
-  public static org.apache.accumulo.core.client.Instance getInstance(Class<?> implementingClass,
-      Configuration conf) {
-    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE), "");
-    if ("ZooKeeperInstance".equals(instanceType)) {
-      return new org.apache.accumulo.core.client.ZooKeeperInstance(
-          getClientConfiguration(implementingClass, conf));
-    } else if (instanceType.isEmpty()) {
-      throw new IllegalStateException(
-          "Instance has not been configured for " + implementingClass.getSimpleName());
-    } else {
-      throw new IllegalStateException("Unrecognized instance type " + instanceType);
-    }
-  }
-
-  /**
-   * Obtain a ClientConfiguration based on the configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   *
-   * @return A ClientConfiguration
-   * @since 1.7.0
-   */
-  public static org.apache.accumulo.core.client.ClientConfiguration
-      getClientConfiguration(Class<?> implementingClass, Configuration conf) {
-    String clientConfigString =
-        conf.get(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG));
-    if (null != clientConfigString) {
-      return org.apache.accumulo.core.client.ClientConfiguration.deserialize(clientConfigString);
-    }
-
-    String instanceName = conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME));
-    String zookeepers = conf.get(enumToConfKey(implementingClass, InstanceOpts.ZOO_KEEPERS));
-    org.apache.accumulo.core.client.ClientConfiguration clientConf =
-        org.apache.accumulo.core.client.ClientConfiguration.loadDefault();
-    if (null != instanceName) {
-      clientConf.withInstance(instanceName);
-    }
-    if (null != zookeepers) {
-      clientConf.withZkHosts(zookeepers);
-    }
-    return clientConf;
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param level the logging level
-   * @since 1.6.0
-   */
-  public static void setLogLevel(Class<?> implementingClass, Configuration conf, Level level) {
-    checkArgument(level != null, "level is null");
-    Logger.getLogger(implementingClass).setLevel(level);
-    conf.setInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), level.toInt());
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @return the log level
-   * @since 1.6.0
-   * @see #setLogLevel(Class, Configuration, Level)
-   */
-  public static Level getLogLevel(Class<?> implementingClass, Configuration conf) {
-    return Level.toLevel(
-        conf.getInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), Level.INFO.toInt()));
-  }
-
-  /**
-   * Sets the valid visibility count for this job.
-   *
-   * @param conf the Hadoop configuration object to configure
-   * @param visibilityCacheSize the LRU cache size
-   */
-  public static void setVisibilityCacheSize(Configuration conf, int visibilityCacheSize) {
-    conf.setInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE), visibilityCacheSize);
-  }
-
-  /**
-   * Gets the valid visibility count for this job.
-   *
-   * @param conf the Hadoop configuration object to configure
-   * @return the valid visibility count
-   */
-  public static int getVisibilityCacheSize(Configuration conf) {
-    return conf.getInt(enumToConfKey(GeneralOpts.VISIBILITY_CACHE_SIZE),
-        Constants.DEFAULT_VISIBILITY_CACHE_SIZE);
-  }
-
-  /**
-   * Unwraps the provided {@link AuthenticationToken} if it is an instance of DelegationTokenStub,
-   * reconstituting it from the provided {@link JobConf}.
-   *
-   * @param job The job
-   * @param token The authentication token
-   */
-  public static AuthenticationToken unwrapAuthenticationToken(JobConf job,
-      AuthenticationToken token) {
-    requireNonNull(job);
-    requireNonNull(token);
-    if (token instanceof org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) {
-      org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub delTokenStub =
-          (org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) token;
-      Token<? extends TokenIdentifier> hadoopToken =
-          job.getCredentials().getToken(new Text(delTokenStub.getServiceName()));
-      AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
-      try {
-        identifier
-            .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
-      } catch (IOException e) {
-        throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
-            e);
-      }
-    }
-    return token;
-  }
-
-  /**
-   * Unwraps the provided {@link AuthenticationToken} if it is an instance of DelegationTokenStub,
-   * reconstituting it from the provided {@link JobConf}.
-   *
-   * @param job The job
-   * @param token The authentication token
-   */
-  public static AuthenticationToken unwrapAuthenticationToken(JobContext job,
-      AuthenticationToken token) {
-    requireNonNull(job);
-    requireNonNull(token);
-    if (token instanceof org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) {
-      org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub delTokenStub =
-          (org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub) token;
-      Token<? extends TokenIdentifier> hadoopToken =
-          job.getCredentials().getToken(new Text(delTokenStub.getServiceName()));
-      AuthenticationTokenIdentifier identifier = new AuthenticationTokenIdentifier();
-      try {
-        identifier
-            .readFields(new DataInputStream(new ByteArrayInputStream(hadoopToken.getIdentifier())));
-        return new DelegationTokenImpl(hadoopToken.getPassword(), identifier);
-      } catch (IOException e) {
-        throw new RuntimeException("Could not construct DelegationToken from JobConf Credentials",
-            e);
-      }
-    }
-    return token;
-  }
-
-  public static ClientContext client(Class<?> CLASS, Configuration conf)
-      throws AccumuloException, AccumuloSecurityException {
-    return ((org.apache.accumulo.core.clientImpl.ConnectorImpl) getInstance(CLASS, conf)
-        .getConnector(getPrincipal(CLASS, conf), getAuthenticationToken(CLASS, conf)))
-        .getAccumuloClient();
-  }
-
-  public static ClientContext client(Class<?> CLASS,
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit split, Configuration conf)
-      throws IOException {
-    try {
-      org.apache.accumulo.core.client.Instance instance =
-          split.getInstance(getClientConfiguration(CLASS, conf));
-      if (instance == null) {
-        instance = getInstance(CLASS, conf);
-      }
-
-      String principal = split.getPrincipal();
-      if (principal == null) {
-        principal = getPrincipal(CLASS, conf);
-      }
-
-      AuthenticationToken token = split.getToken();
-      if (token == null) {
-        token = getAuthenticationToken(CLASS, conf);
-      }
-
-      return ((org.apache.accumulo.core.clientImpl.ConnectorImpl) instance.getConnector(principal,
-          token)).getAccumuloClient();
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new IOException(e);
-    }
-
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java
deleted file mode 100644
index c9a622f..0000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/FileOutputConfigurator.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
-
-import java.util.Arrays;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * @since 1.6.0
- */
-public class FileOutputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link AccumuloConfiguration}.
-   *
-   * @since 1.6.0
-   */
-  public static enum Opts {
-    ACCUMULO_PROPERTIES
-  }
-
-  /**
-   * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the
-   * RecordWriter.<br>
-   * These properties correspond to the supported public static setter methods available to this
-   * class.
-   *
-   * @param property the Accumulo property to check
-   * @since 1.6.0
-   */
-  protected static Boolean isSupportedAccumuloProperty(Property property) {
-    switch (property) {
-      case TABLE_FILE_COMPRESSION_TYPE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
-      case TABLE_FILE_BLOCK_SIZE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
-      case TABLE_FILE_REPLICATION:
-        return true;
-      default:
-        return false;
-    }
-  }
-
-  /**
-   * Helper for transforming Accumulo configuration properties into something that can be stored
-   * safely inside the Hadoop Job configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param property the supported Accumulo property
-   * @param value the value of the property to set
-   * @since 1.6.0
-   */
-  private static <T> void setAccumuloProperty(Class<?> implementingClass, Configuration conf,
-      Property property, T value) {
-    if (isSupportedAccumuloProperty(property)) {
-      String val = String.valueOf(value);
-      if (property.getType().isValidFormat(val)) {
-        conf.set(
-            enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + property.getKey(),
-            val);
-      } else {
-        throw new IllegalArgumentException(
-            "Value is not appropriate for property type '" + property.getType() + "'");
-      }
-    } else {
-      throw new IllegalArgumentException("Unsupported configuration property " + property.getKey());
-    }
-  }
-
-  /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo
-   * defaults, and overridden with Accumulo properties that have been stored in the Job's
-   * configuration.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @since 1.6.0
-   */
-  public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass,
-      Configuration conf) {
-    String prefix = enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + ".";
-    ConfigurationCopy acuConf = new ConfigurationCopy(DefaultConfiguration.getInstance());
-    for (Entry<String,String> entry : conf) {
-      if (entry.getKey().startsWith(prefix)) {
-        String propString = entry.getKey().substring(prefix.length());
-        Property prop = Property.getPropertyByKey(propString);
-        if (prop != null) {
-          acuConf.set(prop, entry.getValue());
-        } else if (Property.isValidTablePropertyKey(propString)) {
-          acuConf.set(propString, entry.getValue());
-        } else {
-          throw new IllegalArgumentException("Unknown accumulo file property " + propString);
-        }
-      }
-    }
-    return acuConf;
-  }
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require
-   * additional libraries to be available to your Job.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param compressionType one of "none", "gz", "bzip2", "lzo", "lz4", "snappy", or "zstd"
-   * @since 1.6.0
-   */
-  public static void setCompressionType(Class<?> implementingClass, Configuration conf,
-      String compressionType) {
-    if (compressionType == null || !Arrays
-        .asList("none", "gz", "bzip2", "lzo", "lz4", "snappy", "zstd").contains(compressionType)) {
-      throw new IllegalArgumentException(
-          "Compression type must be one of: none, gz, bzip2, lzo, lz4, snappy, zstd");
-    }
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSION_TYPE,
-        compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as
-   * a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size
-   * of the indexes (which can also affect seek performance).
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param dataBlockSize the block size, in bytes
-   * @since 1.6.0
-   */
-  public static void setDataBlockSize(Class<?> implementingClass, Configuration conf,
-      long dataBlockSize) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE,
-        dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by
-   * the underlying file system.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param fileBlockSize the block size, in bytes
-   * @since 1.6.0
-   */
-  public static void setFileBlockSize(Class<?> implementingClass, Configuration conf,
-      long fileBlockSize) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_BLOCK_SIZE, fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy
-   * within the file, while larger blocks mean a more shallow index hierarchy within the file. This
-   * can affect the performance of queries.
-   *
-   * @param implementingClass the class whose name will be used as a prefix for the property
-   *        configuration key
-   * @param conf the Hadoop configuration object to configure
-   * @param indexBlockSize the block size, in bytes
-   * @since 1.6.0
-   */
-  public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf,
-      long indexBlockSize) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX,