You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/12/19 06:53:25 UTC

[GitHub] ctubbsii closed pull request #804: Make old MR more backwards compatible & deprecate

ctubbsii closed pull request #804:  Make old MR more backwards compatible & deprecate
URL: https://github.com/apache/accumulo/pull/804
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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
index 32af1a0dfd..60f824e8b1 100644
--- 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
@@ -28,9 +28,9 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -47,7 +47,6 @@
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 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;
@@ -55,16 +54,13 @@
 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.ClientInfo;
 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.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapred.BatchInputSplit;
-import org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
-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;
@@ -85,8 +81,17 @@
  * 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
 public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
+
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator {}
+
   protected static final Class<?> CLASS = AccumuloInputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
 
@@ -100,7 +105,7 @@
    * @since 1.8.0
    */
   public static void setClassLoaderContext(JobConf job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job, context);
+    Configurator.setClassLoaderContext(CLASS, job, context);
   }
 
   /**
@@ -112,46 +117,7 @@ public static void setClassLoaderContext(JobConf job, String context) {
    * @since 1.8.0
    */
   public static String getClassLoaderContext(JobConf job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job);
-  }
-
-  /**
-   * Sets connection information needed to communicate with Accumulo for this job
-   *
-   * @param job
-   *          Hadoop job instance to be configured
-   * @param clientProps
-   *          Connection information for Accumulo
-   * @since 2.0.0
-   */
-  public static void setClientProperties(JobConf job, Properties clientProps) {
-    Properties inputProps = InputConfigurator.updateToken(job.getCredentials(), clientProps);
-    InputConfigurator.setClientProperties(CLASS, job, inputProps);
-  }
-
-  /**
-   * Set Accumulo client properties file used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientPropsFile
-   *          URL to Accumulo client properties file
-   * @since 2.0.0
-   */
-  public static void setClientPropertiesFile(JobConf job, String clientPropsFile) {
-    InputConfigurator.setClientPropertiesFile(CLASS, job, clientPropsFile);
-  }
-
-  /**
-   * Retrieves {@link Properties} from the configuration
-   *
-   * @param job
-   *          Hadoop job instance configuration
-   * @return {@link Properties} object
-   * @since 2.0.0
-   */
-  protected static Properties getClientProperties(JobConf job) {
-    return InputConfigurator.getClientProperties(CLASS, job);
+    return Configurator.getClassLoaderContext(CLASS, job);
   }
 
   /**
@@ -173,15 +139,13 @@ protected static Properties getClientProperties(JobConf job) {
    * @param token
    *          the user's password
    * @since 1.5.0
-   * @deprecated since 2.0.0, use {@link #setClientProperties(JobConf, Properties)} instead
    */
-  @Deprecated
   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 (AccumuloClient client = Accumulo.newClient().from(getClientProperties(job))
-          .as(principal, token).build()) {
+      try (AccumuloClient client = Accumulo.newClient()
+          .from(Configurator.getClientProperties(CLASS, job)).as(principal, token).build()) {
         token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
       } catch (Exception e) {
         log.warn("Failed to automatically obtain DelegationToken, Mappers/Reducers will likely"
@@ -202,7 +166,7 @@ public static void setConnectorInfo(JobConf job, String principal, Authenticatio
       job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
     }
 
-    InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
+    Configurator.setConnectorInfo(CLASS, job, principal, token);
   }
 
   /**
@@ -219,12 +183,10 @@ public static void setConnectorInfo(JobConf job, String principal, Authenticatio
    * @param tokenFile
    *          the path to the token file
    * @since 1.6.0
-   * @deprecated since 2.0.0, use {@link #setClientPropertiesFile(JobConf, String)} instead
    */
-  @Deprecated
   public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
       throws AccumuloSecurityException {
-    setClientPropertiesFile(job, tokenFile);
+    Configurator.setConnectorInfo(CLASS, job, principal, tokenFile);
   }
 
   /**
@@ -237,7 +199,7 @@ public static void setConnectorInfo(JobConf job, String principal, String tokenF
    * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
    */
   protected static Boolean isConnectorInfoSet(JobConf job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job);
+    return Configurator.isConnectorInfoSet(CLASS, job);
   }
 
   /**
@@ -250,7 +212,7 @@ protected static Boolean isConnectorInfoSet(JobConf job) {
    * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
    */
   protected static String getPrincipal(JobConf job) {
-    return InputConfigurator.getPrincipal(CLASS, job);
+    return Configurator.getPrincipal(CLASS, job);
   }
 
   /**
@@ -265,8 +227,28 @@ protected static String getPrincipal(JobConf job) {
    * @see #setConnectorInfo(JobConf, String, String)
    */
   protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    AuthenticationToken token = InputConfigurator.getAuthenticationToken(CLASS, job);
-    return ConfiguratorBase.unwrapAuthenticationToken(job, token);
+    AuthenticationToken token = Configurator.getAuthenticationToken(CLASS, job);
+    return Configurator.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
+  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
+    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
+        .withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
@@ -277,12 +259,10 @@ protected static AuthenticationToken getAuthenticationToken(JobConf job) {
    * @param clientConfig
    *          client configuration containing connection options
    * @since 1.6.0
-   * @deprecated since 2.0.0; Use {@link #setClientProperties(JobConf, Properties)} instead.
    */
-  @Deprecated
   public static void setZooKeeperInstance(JobConf job,
       org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
+    Configurator.setZooKeeperInstance(CLASS, job, clientConfig);
   }
 
   /**
@@ -293,11 +273,10 @@ public static void setZooKeeperInstance(JobConf job,
    *          the Hadoop context for the configured job
    * @return an Accumulo instance
    * @since 1.5.0
-   * @deprecated since 2.0.0, Use {@link #getClientProperties(JobConf)} instead
+   * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
    */
-  @Deprecated
   protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
-    return InputConfigurator.getInstance(CLASS, job);
+    return Configurator.getInstance(CLASS, job);
   }
 
   /**
@@ -310,7 +289,7 @@ public static void setZooKeeperInstance(JobConf job,
    * @since 1.5.0
    */
   public static void setLogLevel(JobConf job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job, level);
+    Configurator.setLogLevel(CLASS, job, level);
   }
 
   /**
@@ -323,12 +302,12 @@ public static void setLogLevel(JobConf job, Level level) {
    * @see #setLogLevel(JobConf, Level)
    */
   protected static Level getLogLevel(JobConf job) {
-    return InputConfigurator.getLogLevel(CLASS, job);
+    return Configurator.getLogLevel(CLASS, job);
   }
 
   /**
-   * Sets the {@link org.apache.accumulo.core.security.Authorizations} used to scan. Must be a
-   * subset of the user's authorization. Defaults to the empty set.
+   * 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
@@ -337,7 +316,7 @@ protected static Level getLogLevel(JobConf job) {
    * @since 1.5.0
    */
   public static void setScanAuthorizations(JobConf job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
+    Configurator.setScanAuthorizations(CLASS, job, auths);
   }
 
   /**
@@ -350,7 +329,7 @@ public static void setScanAuthorizations(JobConf job, Authorizations auths) {
    * @see #setScanAuthorizations(JobConf, Authorizations)
    */
   protected static Authorizations getScanAuthorizations(JobConf job) {
-    return InputConfigurator.getScanAuthorizations(CLASS, job);
+    return Configurator.getScanAuthorizations(CLASS, job);
   }
 
   /**
@@ -360,12 +339,10 @@ protected static Authorizations getScanAuthorizations(JobConf job) {
    *          The job
    * @return The client configuration for the job
    * @since 1.7.0
-   * @deprecated since 2.0.0, replaced by {@link #getClientProperties(JobConf)}
    */
-  @Deprecated
   protected static org.apache.accumulo.core.client.ClientConfiguration getClientConfiguration(
       JobConf job) {
-    return InputConfigurator.getClientConfiguration(CLASS, job);
+    return Configurator.getClientConfiguration(CLASS, job);
   }
 
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
@@ -380,24 +357,27 @@ protected static Authorizations getScanAuthorizations(JobConf job) {
    * @since 1.5.0
    */
   protected static void validateOptions(JobConf job) throws IOException {
-    AccumuloClient client = InputConfigurator.getClient(CLASS, job);
-    InputConfigurator.validatePermissions(CLASS, job, client);
+    Configurator.validatePermissions(CLASS, job);
   }
 
   /**
-   * Fetches all {@link InputTableConfig}s that have been set on the given Hadoop 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 InputTableConfig} objects set on the job
+   * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects set on
+   *         the job
    * @since 1.6.0
    */
-  public static Map<String,InputTableConfig> getInputTableConfigs(JobConf job) {
-    return InputConfigurator.getInputTableConfigs(CLASS, job);
+  public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> getInputTableConfigs(
+      JobConf job) {
+    return Configurator.getInputTableConfigs(CLASS, job);
   }
 
   /**
-   * Fetches a {@link InputTableConfig} that has been set on the configuration for a specific table.
+   * 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.
@@ -406,18 +386,18 @@ protected static void validateOptions(JobConf job) throws IOException {
    *          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
+   * @return the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the given
+   *         table
    * @since 1.6.0
    */
-  public static InputTableConfig getInputTableConfig(JobConf job, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, job, tableName);
+  public static org.apache.accumulo.core.client.mapreduce.InputTableConfig getInputTableConfig(
+      JobConf job, String tableName) {
+    return Configurator.getInputTableConfig(CLASS, job, tableName);
   }
 
   /**
-   * An abstract base class to be used to create {@link org.apache.hadoop.mapred.RecordReader}
-   * instances that convert from Accumulo
-   * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to
-   * the user's K/V types.
+   * 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:
@@ -427,6 +407,7 @@ public static InputTableConfig getInputTableConfig(JobConf job, String tableName
    * </ul>
    */
   protected abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
+    private ClientContext context = null;
     protected long numKeysRead;
     protected Iterator<Map.Entry<Key,Value>> scannerIterator;
     protected RangeInputSplit split;
@@ -473,6 +454,24 @@ private void setupIterators(JobConf job, ScannerBase scanner, String tableName,
         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
+    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.
      */
@@ -480,130 +479,139 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
       baseSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + baseSplit);
 
-      try (ClientContext context = new ClientContext(getClientProperties(job))) {
-        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
-        InputTableConfig tableConfig = getInputTableConfig(job, baseSplit.getTableName());
-
-        log.debug("Created client with user: " + context.whoami());
-        log.debug("Creating scanner for table: " + table);
-        log.debug("Authorizations are: " + authorizations);
-
-        if (baseSplit instanceof BatchInputSplit) {
-          BatchScanner scanner;
-          BatchInputSplit multiRangeSplit = (BatchInputSplit) baseSplit;
-
-          try {
-            // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-            // will not span tablets
-            int scanThreads = 1;
-            scanner = context.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();
+      if (context == null) {
+        context = new ClientContext(Configurator.getClientProperties(CLASS, job));
+      }
+      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());
+
+      log.debug("Created client with user: " + context.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 = context.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 scanner;
+        scanner.setRanges(multiRangeSplit.getRanges());
+        scannerBase = scanner;
 
-          try {
-            if (isOffline) {
-              scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
-                  authorizations);
-            } else {
-              scanner = new ScannerImpl(context, Table.ID.of(baseSplit.getTableId()),
-                  authorizations);
-            }
-            if (isIsolated) {
-              log.info("Creating isolated scanner");
-              scanner = new IsolatedScanner(scanner);
-            }
-            if (usesLocalIterators) {
-              log.info("Using local iterators");
-              scanner = new ClientSideIteratorScanner(scanner);
-            }
-            setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-          } catch (Exception e) {
-            throw new IOException(e);
-          }
+      } else if (baseSplit instanceof RangeInputSplit) {
+        split = (RangeInputSplit) baseSplit;
+        Boolean isOffline = baseSplit.isOffline();
+        if (isOffline == null) {
+          isOffline = tableConfig.isOfflineScan();
+        }
 
-          scanner.setRange(baseSplit.getRange());
-          scannerBase = scanner;
-        } else {
-          throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
+        Boolean isIsolated = baseSplit.isIsolatedScan();
+        if (isIsolated == null) {
+          isIsolated = tableConfig.shouldUseIsolatedScanners();
         }
 
-        Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
-        if (columns == null) {
-          columns = tableConfig.getFetchedColumns();
+        Boolean usesLocalIterators = baseSplit.usesLocalIterators();
+        if (usesLocalIterators == null) {
+          usesLocalIterators = tableConfig.shouldUseLocalIterators();
         }
 
-        // 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());
+        Scanner scanner;
+
+        try {
+          if (isOffline) {
+            scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
+                authorizations);
           } else {
-            log.debug("Fetching column family " + c.getFirst());
-            scannerBase.fetchColumnFamily(c.getFirst());
+            scanner = new ScannerImpl(context, Table.ID.of(baseSplit.getTableId()), authorizations);
+          }
+          if (isIsolated) {
+            log.info("Creating isolated scanner");
+            scanner = new IsolatedScanner(scanner);
           }
+          if (usesLocalIterators) {
+            log.info("Using local iterators");
+            scanner = new ClientSideIteratorScanner(scanner);
+          }
+          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
+        } catch (Exception e) {
+          throw new IOException(e);
         }
 
-        SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
-        if (samplerConfig == null) {
-          samplerConfig = tableConfig.getSamplerConfiguration();
-        }
+        scanner.setRange(baseSplit.getRange());
+        scannerBase = scanner;
+      } else {
+        throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
+      }
 
-        if (samplerConfig != null) {
-          scannerBase.setSamplerConfiguration(samplerConfig);
-        }
+      Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
+      if (columns == null) {
+        columns = tableConfig.getFetchedColumns();
+      }
 
-        Map<String,String> executionHints = baseSplit.getExecutionHints();
-        if (executionHints == null || executionHints.size() == 0) {
-          executionHints = tableConfig.getExecutionHints();
+      // 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());
         }
+      }
 
-        if (executionHints != null) {
-          scannerBase.setExecutionHints(executionHints);
-        }
+      SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
+      if (samplerConfig == null) {
+        samplerConfig = tableConfig.getSamplerConfiguration();
+      }
 
-        scannerIterator = scannerBase.iterator();
-        numKeysRead = 0;
+      if (samplerConfig != null) {
+        scannerBase.setSamplerConfiguration(samplerConfig);
       }
+
+      scannerIterator = scannerBase.iterator();
+      numKeysRead = 0;
     }
 
     @Override
     public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
+      // close several objects, aggregating any exceptions thrown
+      Stream.of(scannerBase, context).flatMap(o -> {
+        try {
+          if (o != null) {
+            o.close();
+          }
+          return null;
+        } catch (Exception e) {
+          return Stream.of(e);
+        }
+      }).reduce((e1, e2) -> {
+        e1.addSuppressed(e2);
+        return e1;
+      }).ifPresent(e -> {
+        if (e instanceof RuntimeException) {
+          throw (RuntimeException) e;
+        } else {
+          throw new RuntimeException(e);
+        }
+      });
     }
 
     @Override
@@ -625,8 +633,8 @@ public float getProgress() throws IOException {
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, Table.ID tableId,
       List<Range> ranges)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    ClientContext context = new ClientContext(getClientProperties(job));
-    return InputConfigurator.binOffline(tableId, ranges, context);
+    return Configurator.binOffline(tableId, ranges,
+        ClientInfo.from(Configurator.getClientProperties(CLASS, job)));
   }
 
   /**
@@ -646,138 +654,146 @@ public float getProgress() throws IOException {
 
     Random random = new SecureRandom();
     LinkedList<InputSplit> splits = new LinkedList<>();
-    Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(job);
-    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
+    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();
-      InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-      ClientContext context = new ClientContext(getClientProperties(job));
-      Table.ID tableId;
-      // resolve table name to id once, and use id from this point forward
-      try {
-        tableId = Tables.getTableId(context, tableName);
-      } catch (TableNotFoundException e) {
-        throw new IOException(e);
-      }
+      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig = tableConfigEntry
+          .getValue();
+
+      try (
+          ClientContext context = new ClientContext(Configurator.getClientProperties(CLASS, job))) {
+        Table.ID tableId;
+        // resolve table name to id once, and use id from this point forward
+        try {
+          tableId = Tables.getTableId(context, 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());
-      }
+        boolean batchScan = Configurator.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);
+        // 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);
-          }
-        } else {
-          tl = InputConfigurator.getTabletLocator(CLASS, job, tableId);
-          // its possible that the cache could contain complete, but old information about a tables
-          // tablets... so clear it
-          tl.invalidateCache();
-
-          while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
-            String tableIdStr = tableId.canonicalID();
-            if (!Tables.exists(context, tableId))
-              throw new TableDeletedException(tableIdStr);
-            if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
-              throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
-            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);
+            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(context, tableId);
+            // its possible that the cache could contain complete, but old information about a
+            // tables tablets... so clear it
             tl.invalidateCache();
+
+            while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
+              String tableIdStr = tableId.canonicalID();
+              if (!Tables.exists(context, tableId))
+                throw new TableDeletedException(tableIdStr);
+              if (Tables.getTableState(context, tableId) == TableState.OFFLINE)
+                throw new TableOfflineException(Tables.getTableOfflineMsg(context, tableId));
+              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);
         }
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
 
-      HashMap<Range,ArrayList<String>> splitsToAdd = null;
+        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));
-
-            BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges,
-                new String[] {location});
-            SplitUtils.updateSplit(split, tableConfig, logLevel);
+        if (!autoAdjust)
+          splitsToAdd = new HashMap<>();
 
-            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.canonicalID(),
-                    ke.clip(r), new String[] {location});
-                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);
+        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.canonicalID(),
+                      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.canonicalID(),
-              entry.getKey(), entry.getValue().toArray(new String[0]));
-          SplitUtils.updateSplit(split, tableConfig, logLevel);
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+        if (!autoAdjust)
+          for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
+            RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                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);
-        }
+            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
index 0ab83c360a..2074c8cf3a 100644
--- 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
@@ -21,10 +21,6 @@
 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.client.summary.Summarizer;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
-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;
@@ -51,9 +47,17 @@
  * {@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
 public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator {}
+
   private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
 
@@ -68,7 +72,7 @@
    * @since 1.5.0
    */
   public static void setCompressionType(JobConf job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job, compressionType);
+    Configurator.setCompressionType(CLASS, job, compressionType);
   }
 
   /**
@@ -87,7 +91,7 @@ public static void setCompressionType(JobConf job, String compressionType) {
    * @since 1.5.0
    */
   public static void setDataBlockSize(JobConf job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job, dataBlockSize);
+    Configurator.setDataBlockSize(CLASS, job, dataBlockSize);
   }
 
   /**
@@ -101,7 +105,7 @@ public static void setDataBlockSize(JobConf job, long dataBlockSize) {
    * @since 1.5.0
    */
   public static void setFileBlockSize(JobConf job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job, fileBlockSize);
+    Configurator.setFileBlockSize(CLASS, job, fileBlockSize);
   }
 
   /**
@@ -116,7 +120,7 @@ public static void setFileBlockSize(JobConf job, long fileBlockSize) {
    * @since 1.5.0
    */
   public static void setIndexBlockSize(JobConf job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job, indexBlockSize);
+    Configurator.setIndexBlockSize(CLASS, job, indexBlockSize);
   }
 
   /**
@@ -130,7 +134,7 @@ public static void setIndexBlockSize(JobConf job, long indexBlockSize) {
    * @since 1.5.0
    */
   public static void setReplication(JobConf job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job, replication);
+    Configurator.setReplication(CLASS, job, replication);
   }
 
   /**
@@ -145,21 +149,7 @@ public static void setReplication(JobConf job, int replication) {
    */
 
   public static void setSampler(JobConf job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job, samplerConfig);
-  }
-
-  /**
-   * Specifies a list of summarizer configurations to create summary data in the output file. Each
-   * Key Value written will be passed to the configured {@link Summarizer}'s.
-   *
-   * @param job
-   *          The Hadoop job instance to be configured
-   * @param summarizerConfigs
-   *          summarizer configurations
-   * @since 2.0.0
-   */
-  public static void setSummarizers(JobConf job, SummarizerConfiguration... summarizerConfigs) {
-    FileOutputConfigurator.setSummarizers(CLASS, job, summarizerConfigs);
+    Configurator.setSampler(CLASS, job, samplerConfig);
   }
 
   @Override
@@ -167,13 +157,12 @@ public static void setSummarizers(JobConf job, SummarizerConfiguration... summar
       Progressable progress) throws IOException {
     // get the path of the temporary output file
     final Configuration conf = job;
-    final AccumuloConfiguration acuConf = FileOutputConfigurator.getAccumuloConfiguration(CLASS,
-        job);
+    final AccumuloConfiguration acuConf = Configurator.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 = ConfiguratorBase.getVisibilityCacheSize(conf);
+    final int visCacheSize = Configurator.getVisibilityCacheSize(conf);
 
     return new RecordWriter<Key,Value>() {
       RFileWriter out = null;
@@ -195,5 +184,4 @@ public void write(Key key, Value value) throws IOException {
       }
     };
   }
-
 }
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
index 7d8188450f..241bf6189e 100644
--- 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
@@ -19,6 +19,7 @@
 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;
@@ -37,13 +38,19 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloInputFormat#setClientProperties(JobConf, java.util.Properties)}
- * <li>{@link AccumuloInputFormat#setInputTableName(JobConf, String)}</li>
+ * <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
 public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
 
   @Override
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
index cf19819d3f..8ea4f56737 100644
--- 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
@@ -19,11 +19,10 @@
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.mapred.InputFormatBase.RecordReaderBase;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
+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;
@@ -38,18 +37,24 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloInputFormat#setClientProperties(JobConf, java.util.Properties)}
- * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, org.apache.accumulo.core.security.Authorizations)}
+ * <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
 public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
 
   /**
-   * Sets the {@link InputTableConfig} objects on the given Hadoop configuration
+   * 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
@@ -57,15 +62,17 @@
    *          the table query configs to be set on the configuration.
    * @since 1.6.0
    */
-  public static void setInputTableConfigs(JobConf job, Map<String,InputTableConfig> configs) {
-    InputConfigurator.setInputTableConfigs(CLASS, job, configs);
+  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));
-    RecordReaderBase<Key,Value> recordReader = new RecordReaderBase<Key,Value>() {
+    InputFormatBase.RecordReaderBase<Key,Value> recordReader = new InputFormatBase.RecordReaderBase<Key,Value>() {
 
       @Override
       public boolean next(Key key, Value value) throws IOException {
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
index 478f85148b..921f2916aa 100644
--- 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
@@ -20,7 +20,6 @@
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -37,12 +36,12 @@
 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.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
-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.data.TabletId;
@@ -66,54 +65,26 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloOutputFormat#setClientProperties(JobConf, Properties)}
+ * <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
 public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator {}
+
   private static final Class<?> CLASS = AccumuloOutputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
 
-  /**
-   * Set the connection information needed to communicate with Accumulo in this job.
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientProps
-   *          Accumulo connection information
-   * @since 2.0.0
-   */
-  public static void setClientProperties(JobConf job, Properties clientProps) {
-    Properties outProps = OutputConfigurator.updateToken(job.getCredentials(), clientProps);
-    OutputConfigurator.setClientProperties(CLASS, job, outProps);
-  }
-
-  /**
-   * Get the connection information needed to communication with Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @since 2.0.0
-   */
-  protected static Properties getClientProperties(JobConf job) {
-    return OutputConfigurator.getClientProperties(CLASS, job);
-  }
-
-  /**
-   * Set Accumulo client properties file used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientPropsFile
-   *          URL (hdfs:// or http://) to Accumulo client properties file
-   * @since 2.0.0
-   */
-  public static void setClientPropertiesFile(JobConf job, String clientPropsFile) {
-    OutputConfigurator.setClientPropertiesFile(CLASS, job, clientPropsFile);
-  }
-
   /**
    * Sets the connector information needed to communicate with Accumulo in this job.
    *
@@ -134,11 +105,20 @@ public static void setClientPropertiesFile(JobConf job, String clientPropsFile)
    * @param token
    *          the user's password
    * @since 1.5.0
-   * @deprecated since 2.0.0, use {@link #setClientProperties(JobConf, Properties)} instead.
    */
-  @Deprecated
   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 {
+        org.apache.accumulo.core.client.Instance instance = getInstance(job);
+        org.apache.accumulo.core.client.Connector conn = instance.getConnector(principal, token);
+        token = conn.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) {
@@ -153,7 +133,7 @@ public static void setConnectorInfo(JobConf job, String principal, Authenticatio
       job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
     }
 
-    OutputConfigurator.setConnectorInfo(CLASS, job, principal, token);
+    Configurator.setConnectorInfo(CLASS, job, principal, token);
   }
 
   /**
@@ -171,12 +151,10 @@ public static void setConnectorInfo(JobConf job, String principal, Authenticatio
    * @param tokenFile
    *          the path to the password file
    * @since 1.6.0
-   * @deprecated since 2.0.0, use {@link #setClientPropertiesFile(JobConf, String)} instead
    */
-  @Deprecated
   public static void setConnectorInfo(JobConf job, String principal, String tokenFile)
       throws AccumuloSecurityException {
-    setClientPropertiesFile(job, tokenFile);
+    Configurator.setConnectorInfo(CLASS, job, principal, tokenFile);
   }
 
   /**
@@ -189,7 +167,7 @@ public static void setConnectorInfo(JobConf job, String principal, String tokenF
    * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
    */
   protected static Boolean isConnectorInfoSet(JobConf job) {
-    return OutputConfigurator.isConnectorInfoSet(CLASS, job);
+    return Configurator.isConnectorInfoSet(CLASS, job);
   }
 
   /**
@@ -202,7 +180,29 @@ protected static Boolean isConnectorInfoSet(JobConf job) {
    * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
    */
   protected static String getPrincipal(JobConf job) {
-    return OutputConfigurator.getPrincipal(CLASS, job);
+    return Configurator.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
+  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
+  protected static byte[] getToken(JobConf job) {
+    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
   }
 
   /**
@@ -217,8 +217,28 @@ protected static String getPrincipal(JobConf job) {
    * @see #setConnectorInfo(JobConf, String, String)
    */
   protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    AuthenticationToken token = OutputConfigurator.getAuthenticationToken(CLASS, job);
-    return ConfiguratorBase.unwrapAuthenticationToken(job, token);
+    AuthenticationToken token = Configurator.getAuthenticationToken(CLASS, job);
+    return Configurator.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
+  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
+    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
+        .withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
@@ -230,12 +250,10 @@ protected static AuthenticationToken getAuthenticationToken(JobConf job) {
    * @param clientConfig
    *          client configuration for specifying connection timeouts, SSL connection options, etc.
    * @since 1.6.0
-   * @deprecated since 2.0.0; Use {@link #setClientProperties(JobConf, Properties)} instead.
    */
-  @Deprecated
   public static void setZooKeeperInstance(JobConf job,
       org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job, clientConfig);
+    Configurator.setZooKeeperInstance(CLASS, job, clientConfig);
   }
 
   /**
@@ -246,11 +264,10 @@ public static void setZooKeeperInstance(JobConf job,
    *          the Hadoop context for the configured job
    * @return an Accumulo instance
    * @since 1.5.0
-   * @deprecated since 2.0.0; Use {@link #getClientProperties(JobConf)} instead
+   * @see #setZooKeeperInstance(JobConf, org.apache.accumulo.core.client.ClientConfiguration)
    */
-  @Deprecated
   protected static org.apache.accumulo.core.client.Instance getInstance(JobConf job) {
-    return OutputConfigurator.getInstance(CLASS, job);
+    return Configurator.getInstance(CLASS, job);
   }
 
   /**
@@ -263,7 +280,7 @@ public static void setZooKeeperInstance(JobConf job,
    * @since 1.5.0
    */
   public static void setLogLevel(JobConf job, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, job, level);
+    Configurator.setLogLevel(CLASS, job, level);
   }
 
   /**
@@ -276,7 +293,7 @@ public static void setLogLevel(JobConf job, Level level) {
    * @see #setLogLevel(JobConf, Level)
    */
   protected static Level getLogLevel(JobConf job) {
-    return OutputConfigurator.getLogLevel(CLASS, job);
+    return Configurator.getLogLevel(CLASS, job);
   }
 
   /**
@@ -290,7 +307,7 @@ protected static Level getLogLevel(JobConf job) {
    * @since 1.5.0
    */
   public static void setDefaultTableName(JobConf job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job, tableName);
+    Configurator.setDefaultTableName(CLASS, job, tableName);
   }
 
   /**
@@ -303,7 +320,7 @@ public static void setDefaultTableName(JobConf job, String tableName) {
    * @see #setDefaultTableName(JobConf, String)
    */
   protected static String getDefaultTableName(JobConf job) {
-    return OutputConfigurator.getDefaultTableName(CLASS, job);
+    return Configurator.getDefaultTableName(CLASS, job);
   }
 
   /**
@@ -318,7 +335,7 @@ protected static String getDefaultTableName(JobConf job) {
    * @since 1.5.0
    */
   public static void setBatchWriterOptions(JobConf job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job, bwConfig);
+    Configurator.setBatchWriterOptions(CLASS, job, bwConfig);
   }
 
   /**
@@ -331,7 +348,7 @@ public static void setBatchWriterOptions(JobConf job, BatchWriterConfig bwConfig
    * @see #setBatchWriterOptions(JobConf, BatchWriterConfig)
    */
   protected static BatchWriterConfig getBatchWriterOptions(JobConf job) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, job);
+    return Configurator.getBatchWriterOptions(CLASS, job);
   }
 
   /**
@@ -348,7 +365,7 @@ protected static BatchWriterConfig getBatchWriterOptions(JobConf job) {
    * @since 1.5.0
    */
   public static void setCreateTables(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job, enableFeature);
+    Configurator.setCreateTables(CLASS, job, enableFeature);
   }
 
   /**
@@ -361,7 +378,7 @@ public static void setCreateTables(JobConf job, boolean enableFeature) {
    * @see #setCreateTables(JobConf, boolean)
    */
   protected static Boolean canCreateTables(JobConf job) {
-    return OutputConfigurator.canCreateTables(CLASS, job);
+    return Configurator.canCreateTables(CLASS, job);
   }
 
   /**
@@ -378,7 +395,7 @@ protected static Boolean canCreateTables(JobConf job) {
    * @since 1.5.0
    */
   public static void setSimulationMode(JobConf job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job, enableFeature);
+    Configurator.setSimulationMode(CLASS, job, enableFeature);
   }
 
   /**
@@ -391,7 +408,7 @@ public static void setSimulationMode(JobConf job, boolean enableFeature) {
    * @see #setSimulationMode(JobConf, boolean)
    */
   protected static Boolean getSimulationMode(JobConf job) {
-    return OutputConfigurator.getSimulationMode(CLASS, job);
+    return Configurator.getSimulationMode(CLASS, job);
   }
 
   /**
@@ -426,7 +443,8 @@ protected AccumuloRecordWriter(JobConf job) {
       this.defaultTableName = (tname == null) ? null : new Text(tname);
 
       if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientProperties(job)).build();
+        this.client = Accumulo.newClient().from(Configurator.getClientProperties(CLASS, job))
+            .build();
         mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(job));
       }
     }
@@ -563,7 +581,8 @@ public void close(Reporter reporter) throws IOException {
   public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
     if (!isConnectorInfoSet(job))
       throw new IOException("Connector info has not been set.");
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties(job)).build()) {
+    try (AccumuloClient c = Accumulo.newClient().from(Configurator.getClientProperties(CLASS, job))
+        .build()) {
       String principal = getPrincipal(job);
       AuthenticationToken token = getAuthenticationToken(job);
       if (!c.securityOperations().authenticateUser(principal, token))
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
index 9e01cde7bf..49e6b46fab 100644
--- 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
@@ -20,6 +20,7 @@
 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;
@@ -40,13 +41,18 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloRowInputFormat#setClientProperties(JobConf, java.util.Properties)}
+ * <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
 public class AccumuloRowInputFormat
     extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
   @Override
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
index dcd5bf5547..fb9c17987c 100644
--- 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
@@ -19,16 +19,16 @@
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 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;
@@ -52,9 +52,17 @@
  * 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
 public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator {}
+
   /**
    * Sets the name of the input table, over which this job will scan.
    *
@@ -65,7 +73,7 @@
    * @since 1.5.0
    */
   public static void setInputTableName(JobConf job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job, tableName);
+    Configurator.setInputTableName(CLASS, job, tableName);
   }
 
   /**
@@ -78,7 +86,7 @@ public static void setInputTableName(JobConf job, String tableName) {
    * @see #setInputTableName(JobConf, String)
    */
   protected static String getInputTableName(JobConf job) {
-    return InputConfigurator.getInputTableName(CLASS, job);
+    return Configurator.getInputTableName(CLASS, job);
   }
 
   /**
@@ -92,7 +100,7 @@ protected static String getInputTableName(JobConf job) {
    * @since 1.5.0
    */
   public static void setRanges(JobConf job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job, ranges);
+    Configurator.setRanges(CLASS, job, ranges);
   }
 
   /**
@@ -107,7 +115,7 @@ public static void setRanges(JobConf job, Collection<Range> ranges) {
    * @see #setRanges(JobConf, Collection)
    */
   protected static List<Range> getRanges(JobConf job) throws IOException {
-    return InputConfigurator.getRanges(CLASS, job);
+    return Configurator.getRanges(CLASS, job);
   }
 
   /**
@@ -123,7 +131,7 @@ public static void setRanges(JobConf job, Collection<Range> ranges) {
    */
   public static void fetchColumns(JobConf job,
       Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
+    Configurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
   }
 
   /**
@@ -136,7 +144,7 @@ public static void fetchColumns(JobConf job,
    * @see #fetchColumns(JobConf, Collection)
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
-    return InputConfigurator.getFetchedColumns(CLASS, job);
+    return Configurator.getFetchedColumns(CLASS, job);
   }
 
   /**
@@ -149,7 +157,7 @@ public static void fetchColumns(JobConf job,
    * @since 1.5.0
    */
   public static void addIterator(JobConf job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job, cfg);
+    Configurator.addIterator(CLASS, job, cfg);
   }
 
   /**
@@ -163,7 +171,7 @@ public static void addIterator(JobConf job, IteratorSetting cfg) {
    * @see #addIterator(JobConf, IteratorSetting)
    */
   protected static List<IteratorSetting> getIterators(JobConf job) {
-    return InputConfigurator.getIterators(CLASS, job);
+    return Configurator.getIterators(CLASS, job);
   }
 
   /**
@@ -182,7 +190,7 @@ public static void addIterator(JobConf job, IteratorSetting cfg) {
    * @since 1.5.0
    */
   public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
+    Configurator.setAutoAdjustRanges(CLASS, job, enableFeature);
   }
 
   /**
@@ -196,7 +204,7 @@ public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
    * @see #setAutoAdjustRanges(JobConf, boolean)
    */
   protected static boolean getAutoAdjustRanges(JobConf job) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, job);
+    return Configurator.getAutoAdjustRanges(CLASS, job);
   }
 
   /**
@@ -212,7 +220,7 @@ protected static boolean getAutoAdjustRanges(JobConf job) {
    * @since 1.5.0
    */
   public static void setScanIsolation(JobConf job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
+    Configurator.setScanIsolation(CLASS, job, enableFeature);
   }
 
   /**
@@ -225,7 +233,7 @@ public static void setScanIsolation(JobConf job, boolean enableFeature) {
    * @see #setScanIsolation(JobConf, boolean)
    */
   protected static boolean isIsolated(JobConf job) {
-    return InputConfigurator.isIsolated(CLASS, job);
+    return Configurator.isIsolated(CLASS, job);
   }
 
   /**
@@ -244,7 +252,7 @@ protected static boolean isIsolated(JobConf job) {
    * @since 1.5.0
    */
   public static void setLocalIterators(JobConf job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
+    Configurator.setLocalIterators(CLASS, job, enableFeature);
   }
 
   /**
@@ -257,7 +265,7 @@ public static void setLocalIterators(JobConf job, boolean enableFeature) {
    * @see #setLocalIterators(JobConf, boolean)
    */
   protected static boolean usesLocalIterators(JobConf job) {
-    return InputConfigurator.usesLocalIterators(CLASS, job);
+    return Configurator.usesLocalIterators(CLASS, job);
   }
 
   /**
@@ -296,7 +304,7 @@ protected static boolean usesLocalIterators(JobConf job) {
    * @since 1.5.0
    */
   public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
+    Configurator.setOfflineTableScan(CLASS, job, enableFeature);
   }
 
   /**
@@ -309,14 +317,14 @@ public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
    * @see #setOfflineTableScan(JobConf, boolean)
    */
   protected static boolean isOfflineScan(JobConf job) {
-    return InputConfigurator.isOfflineScan(CLASS, job);
+    return Configurator.isOfflineScan(CLASS, job);
   }
 
   /**
-   * Controls the use of the {@link org.apache.accumulo.core.client.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)
+   * 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.
@@ -340,12 +348,11 @@ protected static boolean isOfflineScan(JobConf job) {
    * @since 1.7.0
    */
   public static void setBatchScan(JobConf job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job, enableFeature);
+    Configurator.setBatchScan(CLASS, job, enableFeature);
   }
 
   /**
-   * Determines whether a configuration has the {@link org.apache.accumulo.core.client.BatchScanner}
-   * feature enabled.
+   * Determines whether a configuration has the {@link BatchScanner} feature enabled.
    *
    * @param job
    *          the Hadoop context for the configured job
@@ -353,7 +360,7 @@ public static void setBatchScan(JobConf job, boolean enableFeature) {
    * @see #setBatchScan(JobConf, boolean)
    */
   public static boolean isBatchScan(JobConf job) {
-    return InputConfigurator.isBatchScan(CLASS, job);
+    return Configurator.isBatchScan(CLASS, job);
   }
 
   /**
@@ -372,17 +379,7 @@ public static boolean isBatchScan(JobConf job) {
    * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
    */
   public static void setSamplerConfiguration(JobConf job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
-  }
-
-  /**
-   * Set these execution hints on scanners created for input splits. See
-   * {@link ScannerBase#setExecutionHints(java.util.Map)}
-   *
-   * @since 2.0.0
-   */
-  public static void setExecutionHints(JobConf job, Map<String,String> hints) {
-    InputConfigurator.setExecutionHints(CLASS, job, hints);
+    Configurator.setSamplerConfiguration(CLASS, job, samplerConfig);
   }
 
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
@@ -391,5 +388,34 @@ public static void setExecutionHints(JobConf job, Map<String,String> hints) {
     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
+    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
+    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
index 256ef073ea..6e42c9721e 100644
--- 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
@@ -25,7 +25,11 @@
 
 /**
  * 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
 @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
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
index fdb3c63212..7f9aab6460 100644
--- 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
@@ -28,9 +28,9 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
 
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
@@ -61,10 +61,6 @@
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
-import org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit;
-import org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
-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;
@@ -87,9 +83,17 @@
  * 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
 public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator {}
+
   protected static final Class<?> CLASS = AccumuloInputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
 
@@ -103,7 +107,7 @@
    * @since 1.8.0
    */
   public static void setClassLoaderContext(Job job, String context) {
-    InputConfigurator.setClassLoaderContext(CLASS, job.getConfiguration(), context);
+    Configurator.setClassLoaderContext(CLASS, job.getConfiguration(), context);
   }
 
   /**
@@ -115,46 +119,7 @@ public static void setClassLoaderContext(Job job, String context) {
    * @since 1.8.0
    */
   public static String getClassLoaderContext(JobContext job) {
-    return InputConfigurator.getClassLoaderContext(CLASS, job.getConfiguration());
-  }
-
-  /**
-   * Sets connection information needed to communicate with Accumulo for this job
-   *
-   * @param job
-   *          Hadoop job instance to be configured
-   * @param clientProps
-   *          Connection information for Accumulo
-   * @since 2.0.0
-   */
-  public static void setClientProperties(Job job, Properties clientProps) {
-    Properties inputProps = InputConfigurator.updateToken(job.getCredentials(), clientProps);
-    InputConfigurator.setClientProperties(CLASS, job.getConfiguration(), inputProps);
-  }
-
-  /**
-   * Set Accumulo client properties file used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientPropsFile
-   *          URL (hdfs:// or http://) to Accumulo client properties file
-   * @since 2.0.0
-   */
-  public static void setClientPropertiesFile(Job job, String clientPropsFile) {
-    InputConfigurator.setClientPropertiesFile(CLASS, job.getConfiguration(), clientPropsFile);
-  }
-
-  /**
-   * Gets the {@link Properties} from the configuration
-   *
-   * @param context
-   *          Hadoop job context
-   * @return Properties
-   * @since 2.0.0
-   */
-  protected static Properties getClientProperties(JobContext context) {
-    return InputConfigurator.getClientProperties(CLASS, context.getConfiguration());
+    return Configurator.getClassLoaderContext(CLASS, job.getConfiguration());
   }
 
   /**
@@ -176,14 +141,13 @@ protected static Properties getClientProperties(JobContext context) {
    * @param token
    *          the user's password
    * @since 1.5.0
-   * @deprecated since 2.0.0; use {@link #setClientProperties(Job, Properties)} instead.
    */
-  @Deprecated
   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 (AccumuloClient client = Accumulo.newClient().from(getClientProperties(job))
+      try (AccumuloClient client = Accumulo.newClient()
+          .from(Configurator.getClientProperties(CLASS, job.getConfiguration()))
           .as(principal, token).build()) {
         token = client.securityOperations().getDelegationToken(new DelegationTokenConfig());
       } catch (Exception e) {
@@ -205,7 +169,7 @@ public static void setConnectorInfo(Job job, String principal, AuthenticationTok
       job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
     }
 
-    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
+    Configurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
   }
 
   /**
@@ -222,12 +186,10 @@ public static void setConnectorInfo(Job job, String principal, AuthenticationTok
    * @param tokenFile
    *          the path to the token file
    * @since 1.6.0
-   * @deprecated since 2.0.0, use {@link #setClientPropertiesFile(Job, String)}
    */
-  @Deprecated
   public static void setConnectorInfo(Job job, String principal, String tokenFile)
       throws AccumuloSecurityException {
-    setClientPropertiesFile(job, tokenFile);
+    Configurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
   }
 
   /**
@@ -240,7 +202,7 @@ public static void setConnectorInfo(Job job, String principal, String tokenFile)
    * @see #setConnectorInfo(Job, String, AuthenticationToken)
    */
   protected static Boolean isConnectorInfoSet(JobContext context) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, context.getConfiguration());
+    return Configurator.isConnectorInfoSet(CLASS, context.getConfiguration());
   }
 
   /**
@@ -253,7 +215,30 @@ protected static Boolean isConnectorInfoSet(JobContext context) {
    * @see #setConnectorInfo(Job, String, AuthenticationToken)
    */
   protected static String getPrincipal(JobContext context) {
-    return InputConfigurator.getPrincipal(CLASS, context.getConfiguration());
+    return Configurator.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
+  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
+  protected static byte[] getToken(JobContext context) {
+    return AuthenticationToken.AuthenticationTokenSerializer
+        .serialize(getAuthenticationToken(context));
   }
 
   /**
@@ -268,9 +253,27 @@ protected static String getPrincipal(JobContext context) {
    * @see #setConnectorInfo(Job, String, String)
    */
   protected static AuthenticationToken getAuthenticationToken(JobContext context) {
-    AuthenticationToken token = InputConfigurator.getAuthenticationToken(CLASS,
+    AuthenticationToken token = Configurator.getAuthenticationToken(CLASS,
         context.getConfiguration());
-    return ConfiguratorBase.unwrapAuthenticationToken(context, token);
+    return Configurator.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
+   */
+  @Deprecated
+  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
+    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
+        .withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
@@ -282,12 +285,10 @@ protected static AuthenticationToken getAuthenticationToken(JobContext context)
    * @param clientConfig
    *          client configuration containing connection options
    * @since 1.6.0
-   * @deprecated since 2.0.0; Use {@link #setClientProperties(Job, Properties)} instead.
    */
-  @Deprecated
   public static void setZooKeeperInstance(Job job,
       org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
+    Configurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
   }
 
   /**
@@ -298,11 +299,9 @@ public static void setZooKeeperInstance(Job job,
    *          the Hadoop context for the configured job
    * @return an Accumulo instance
    * @since 1.5.0
-   * @deprecated since 2.0.0, use {@link #getClientProperties(JobContext)} instead
    */
-  @Deprecated
   protected static org.apache.accumulo.core.client.Instance getInstance(JobContext context) {
-    return InputConfigurator.getInstance(CLASS, context.getConfiguration());
+    return Configurator.getInstance(CLASS, context.getConfiguration());
   }
 
   /**
@@ -315,7 +314,7 @@ public static void setZooKeeperInstance(Job job,
    * @since 1.5.0
    */
   public static void setLogLevel(Job job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
+    Configurator.setLogLevel(CLASS, job.getConfiguration(), level);
   }
 
   /**
@@ -328,12 +327,12 @@ public static void setLogLevel(Job job, Level level) {
    * @see #setLogLevel(Job, Level)
    */
   protected static Level getLogLevel(JobContext context) {
-    return InputConfigurator.getLogLevel(CLASS, context.getConfiguration());
+    return Configurator.getLogLevel(CLASS, context.getConfiguration());
   }
 
   /**
-   * Sets the {@link org.apache.accumulo.core.security.Authorizations} used to scan. Must be a
-   * subset of the user's authorization. Defaults to the empty set.
+   * 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
@@ -341,7 +340,7 @@ protected static Level getLogLevel(JobContext context) {
    *          the user's authorizations
    */
   public static void setScanAuthorizations(Job job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
+    Configurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
   }
 
   /**
@@ -354,7 +353,7 @@ public static void setScanAuthorizations(Job job, Authorizations auths) {
    * @see #setScanAuthorizations(Job, Authorizations)
    */
   protected static Authorizations getScanAuthorizations(JobContext context) {
-    return InputConfigurator.getScanAuthorizations(CLASS, context.getConfiguration());
+    return Configurator.getScanAuthorizations(CLASS, context.getConfiguration());
   }
 
   /**
@@ -366,7 +365,7 @@ protected static Authorizations getScanAuthorizations(JobContext context) {
    * @since 1.6.0
    */
   protected static Map<String,InputTableConfig> getInputTableConfigs(JobContext context) {
-    return InputConfigurator.getInputTableConfigs(CLASS, context.getConfiguration());
+    return Configurator.getInputTableConfigs(CLASS, context.getConfiguration());
   }
 
   /**
@@ -383,7 +382,7 @@ protected static Authorizations getScanAuthorizations(JobContext context) {
    * @since 1.6.0
    */
   protected static InputTableConfig getInputTableConfig(JobContext context, String tableName) {
-    return InputConfigurator.getInputTableConfig(CLASS, context.getConfiguration(), tableName);
+    return Configurator.getInputTableConfig(CLASS, context.getConfiguration(), tableName);
   }
 
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
@@ -398,30 +397,26 @@ protected static InputTableConfig getInputTableConfig(JobContext context, String
    * @since 1.5.0
    */
   protected static void validateOptions(JobContext context) throws IOException {
-    AccumuloClient client = InputConfigurator.getClient(CLASS, context.getConfiguration());
-    InputConfigurator.validatePermissions(CLASS, context.getConfiguration(), client);
+    Configurator.validatePermissions(CLASS, context.getConfiguration());
   }
 
   /**
-   * Construct the ClientConfiguration given the provided context.
+   * Construct the {@link org.apache.accumulo.core.client.ClientConfiguration} given the provided
+   * context.
    *
    * @param context
    *          The Job
    * @return The ClientConfiguration
    * @since 1.7.0
-   * @deprecated since 2.0.0; use {@link #getClientProperties(JobContext)} instead
    */
-  @Deprecated
   protected static org.apache.accumulo.core.client.ClientConfiguration getClientConfiguration(
       JobContext context) {
-    return InputConfigurator.getClientConfiguration(CLASS, context.getConfiguration());
+    return Configurator.getClientConfiguration(CLASS, context.getConfiguration());
   }
 
   /**
    * An abstract base class to be used to create {@link org.apache.hadoop.mapreduce.RecordReader}
-   * instances that convert from Accumulo
-   * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to
-   * the user's K/V types.
+   * 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>
@@ -432,6 +427,7 @@ protected static void validateOptions(JobContext context) throws IOException {
    * </ul>
    */
   protected abstract static class AbstractRecordReader<K,V> extends RecordReader<K,V> {
+    private ClientContext context = null;
     protected long numKeysRead;
     protected Iterator<Map.Entry<Key,Value>> scannerIterator;
     protected ScannerBase scannerBase;
@@ -482,14 +478,34 @@ private void setupIterators(TaskAttemptContext context, ScannerBase scanner, Str
         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
+    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);
 
-      ClientInfo info = ClientInfo.from(getClientProperties(attempt));
-      ClientContext context = new ClientContext(info);
+      if (context == null) {
+        context = new ClientContext(
+            Configurator.getClientProperties(CLASS, attempt.getConfiguration()));
+      }
       Authorizations authorizations = getScanAuthorizations(attempt);
       String classLoaderContext = getClassLoaderContext(attempt);
       String table = split.getTableName();
@@ -499,12 +515,12 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
       // but the scanner will use the table id resolved at job setup time
       InputTableConfig tableConfig = getInputTableConfig(attempt, split.getTableName());
 
-      log.debug("Creating client with user: " + info.getPrincipal());
+      log.debug("Creating client with user: " + context.whoami());
       log.debug("Creating scanner for table: " + table);
       log.debug("Authorizations are: " + authorizations);
 
-      if (split instanceof BatchInputSplit) {
-        BatchInputSplit batchSplit = (BatchInputSplit) split;
+      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 {
@@ -593,24 +609,32 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
         scannerBase.setSamplerConfiguration(samplerConfig);
       }
 
-      Map<String,String> executionHints = split.getExecutionHints();
-      if (executionHints == null || executionHints.isEmpty()) {
-        executionHints = tableConfig.getExecutionHints();
-      }
-
-      if (executionHints != null) {
-        scannerBase.setExecutionHints(executionHints);
-      }
-
       scannerIterator = scannerBase.iterator();
       numKeysRead = 0;
     }
 
     @Override
     public void close() {
-      if (scannerBase != null) {
-        scannerBase.close();
-      }
+      // close several objects, aggregating any exceptions thrown
+      Stream.of(scannerBase, context).flatMap(o -> {
+        try {
+          if (o != null) {
+            o.close();
+          }
+          return null;
+        } catch (Exception e) {
+          return Stream.of(e);
+        }
+      }).reduce((e1, e2) -> {
+        e1.addSuppressed(e2);
+        return e1;
+      }).ifPresent(e -> {
+        if (e instanceof RuntimeException) {
+          throw (RuntimeException) e;
+        } else {
+          throw new RuntimeException(e);
+        }
+      });
     }
 
     @Override
@@ -650,8 +674,8 @@ public V getCurrentValue() throws IOException, InterruptedException {
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, Table.ID tableId,
       List<Range> ranges)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
-    ClientContext clientContext = new ClientContext(getClientProperties(context));
-    return InputConfigurator.binOffline(tableId, ranges, clientContext);
+    return Configurator.binOffline(tableId, ranges,
+        ClientInfo.from(Configurator.getClientProperties(CLASS, context.getConfiguration())));
   }
 
   /**
@@ -671,142 +695,147 @@ public V getCurrentValue() throws IOException, InterruptedException {
     Random random = new SecureRandom();
     LinkedList<InputSplit> splits = new LinkedList<>();
     Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(context);
-    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-
-      String tableName = tableConfigEntry.getKey();
-      InputTableConfig tableConfig = tableConfigEntry.getValue();
-
-      ClientContext clientContext = new ClientContext(getClientProperties(context));
-      Table.ID tableId;
-      // resolve table name to id once, and use id from this point forward
-      try {
-        tableId = Tables.getTableId(clientContext, tableName);
-      } catch (TableNotFoundException e) {
-        throw new IOException(e);
-      }
+    try (AccumuloClient client = Accumulo.newClient()
+        .from(Configurator.getClientProperties(CLASS, context.getConfiguration())).build()) {
+      for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
 
-      boolean batchScan = InputConfigurator.isBatchScan(CLASS, context.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());
-      }
+        String tableName = tableConfigEntry.getKey();
+        InputTableConfig tableConfig = tableConfigEntry.getValue();
 
-      // get the metadata information for these ranges
-      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
-      TabletLocator tl;
-      try {
-        if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(context, 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);
+        Table.ID tableId;
+        // resolve table name to id once, and use id from this point forward
+        try {
+          tableId = Tables.getTableId((ClientContext) client, tableName);
+        } catch (TableNotFoundException e) {
+          throw new IOException(e);
+        }
+
+        boolean batchScan = Configurator.isBatchScan(CLASS, context.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(context, 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(context, tableId, ranges);
 
-          }
-        } else {
-          tl = InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), tableId);
-          // its possible that the cache could contain complete, but old information about a tables
-          // tablets... so clear it
-          tl.invalidateCache();
-
-          while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
-            String tableIdStr = tableId.canonicalID();
-            if (!Tables.exists(clientContext, tableId))
-              throw new TableDeletedException(tableIdStr);
-            if (Tables.getTableState(clientContext, tableId) == TableState.OFFLINE)
-              throw new TableOfflineException(Tables.getTableOfflineMsg(clientContext, tableId));
-            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);
+            }
+          } else {
+            tl = TabletLocator.getLocator((ClientContext) client, tableId);
+            // its possible that the cache could contain complete, but old information about a
+            // tables tablets... so clear it
             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);
+            while (!tl.binRanges((ClientContext) client, ranges, binnedRanges).isEmpty()) {
+              String tableIdStr = tableId.canonicalID();
+              if (!Tables.exists((ClientContext) client, tableId))
+                throw new TableDeletedException(tableIdStr);
+              if (Tables.getTableState((ClientContext) client, tableId) == TableState.OFFLINE)
+                throw new TableOfflineException(
+                    Tables.getTableOfflineMsg((ClientContext) client, tableId));
+              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);
         }
-        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));
-            BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges,
-                new String[] {location});
-            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.canonicalID(),
-                    ke.clip(r), new String[] {location});
-                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);
+        // 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.canonicalID(),
+                      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.canonicalID(),
-              entry.getKey(), entry.getValue().toArray(new String[0]));
-          SplitUtils.updateSplit(split, tableConfig, logLevel);
-          split.setOffline(tableConfig.isOfflineScan());
-          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+        if (!autoAdjust)
+          for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
+            RangeInputSplit split = new RangeInputSplit(tableName, tableId.canonicalID(),
+                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);
-        }
+            splits.add(split);
+          }
+      }
+      return splits;
     }
-    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
index a73bae1744..2ae54a09ad 100644
--- 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
@@ -21,10 +21,6 @@
 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.client.summary.Summarizer;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
-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;
@@ -49,9 +45,17 @@
  * 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
 public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator {}
+
   private static final Class<?> CLASS = AccumuloFileOutputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
 
@@ -66,7 +70,7 @@
    * @since 1.5.0
    */
   public static void setCompressionType(Job job, String compressionType) {
-    FileOutputConfigurator.setCompressionType(CLASS, job.getConfiguration(), compressionType);
+    Configurator.setCompressionType(CLASS, job.getConfiguration(), compressionType);
   }
 
   /**
@@ -85,7 +89,7 @@ public static void setCompressionType(Job job, String compressionType) {
    * @since 1.5.0
    */
   public static void setDataBlockSize(Job job, long dataBlockSize) {
-    FileOutputConfigurator.setDataBlockSize(CLASS, job.getConfiguration(), dataBlockSize);
+    Configurator.setDataBlockSize(CLASS, job.getConfiguration(), dataBlockSize);
   }
 
   /**
@@ -99,7 +103,7 @@ public static void setDataBlockSize(Job job, long dataBlockSize) {
    * @since 1.5.0
    */
   public static void setFileBlockSize(Job job, long fileBlockSize) {
-    FileOutputConfigurator.setFileBlockSize(CLASS, job.getConfiguration(), fileBlockSize);
+    Configurator.setFileBlockSize(CLASS, job.getConfiguration(), fileBlockSize);
   }
 
   /**
@@ -114,7 +118,7 @@ public static void setFileBlockSize(Job job, long fileBlockSize) {
    * @since 1.5.0
    */
   public static void setIndexBlockSize(Job job, long indexBlockSize) {
-    FileOutputConfigurator.setIndexBlockSize(CLASS, job.getConfiguration(), indexBlockSize);
+    Configurator.setIndexBlockSize(CLASS, job.getConfiguration(), indexBlockSize);
   }
 
   /**
@@ -128,7 +132,7 @@ public static void setIndexBlockSize(Job job, long indexBlockSize) {
    * @since 1.5.0
    */
   public static void setReplication(Job job, int replication) {
-    FileOutputConfigurator.setReplication(CLASS, job.getConfiguration(), replication);
+    Configurator.setReplication(CLASS, job.getConfiguration(), replication);
   }
 
   /**
@@ -143,33 +147,19 @@ public static void setReplication(Job job, int replication) {
    */
 
   public static void setSampler(Job job, SamplerConfiguration samplerConfig) {
-    FileOutputConfigurator.setSampler(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  /**
-   * Specifies a list of summarizer configurations to create summary data in the output file. Each
-   * Key Value written will be passed to the configured {@link Summarizer}'s.
-   *
-   * @param job
-   *          The Hadoop job instance to be configured
-   * @param summarizerConfigs
-   *          summarizer configurations
-   * @since 2.0.0
-   */
-  public static void setSummarizers(Job job, SummarizerConfiguration... summarizerConfigs) {
-    FileOutputConfigurator.setSummarizers(CLASS, job.getConfiguration(), summarizerConfigs);
+    Configurator.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,
+    final AccumuloConfiguration acuConf = Configurator.getAccumuloConfiguration(CLASS,
         context.getConfiguration());
 
     final String extension = acuConf.get(Property.TABLE_FILE_TYPE);
     final Path file = this.getDefaultWorkFile(context, "." + extension);
-    final int visCacheSize = ConfiguratorBase.getVisibilityCacheSize(conf);
+    final int visCacheSize = Configurator.getVisibilityCacheSize(conf);
 
     return new RecordWriter<Key,Value>() {
       RFileWriter out = null;
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
index f9e44687eb..acf6cd1041 100644
--- 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
@@ -19,6 +19,7 @@
 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;
@@ -37,13 +38,18 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloInputFormat#setClientProperties(Job, java.util.Properties)}
+ * <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
 public class AccumuloInputFormat extends InputFormatBase<Key,Value> {
 
   @Override
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
index bc451f3b1d..54fd45c5ca 100644
--- 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
@@ -23,7 +23,7 @@
 import java.util.Map;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
+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;
@@ -41,13 +41,18 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloMultiTableInputFormat#setClientProperties(Job, java.util.Properties)}
+ * <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
 public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
 
   /**
@@ -61,7 +66,8 @@
    */
   public static void setInputTableConfigs(Job job, Map<String,InputTableConfig> configs) {
     requireNonNull(configs);
-    InputConfigurator.setInputTableConfigs(CLASS, job.getConfiguration(), configs);
+    org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setInputTableConfigs(CLASS,
+        job.getConfiguration(), configs);
   }
 
   @Override
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
index 8331b96fa0..82214b393f 100644
--- 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
@@ -20,7 +20,6 @@
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -37,12 +36,12 @@
 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.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
-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.data.TabletId;
@@ -67,55 +66,26 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloOutputFormat#setClientProperties(Job, Properties)}
+ * <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
 public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator {}
+
   private static final Class<?> CLASS = AccumuloOutputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
 
-  /**
-   * Set the connection information needed to communicate with Accumulo in this job.
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientProps
-   *          Accumulo connection information
-   * @since 2.0.0
-   */
-  public static void setClientProperties(Job job, Properties clientProps) {
-    OutputConfigurator.setClientProperties(CLASS, job.getConfiguration(), clientProps);
-  }
-
-  /**
-   * Get connection information from this job
-   *
-   * @param context
-   *          Hadoop job context
-   * @return {@link Properties}
-   *
-   * @since 2.0.0
-   */
-  protected static Properties getClientProperties(JobContext context) {
-    return OutputConfigurator.getClientProperties(CLASS, context.getConfiguration());
-  }
-
-  /**
-   * Set Accumulo client properties file used to connect to Accumulo
-   *
-   * @param job
-   *          Hadoop job to be configured
-   * @param clientPropsFile
-   *          URL to Accumulo client properties file
-   * @since 2.0.0
-   */
-  public static void setClientPropertiesFile(Job job, String clientPropsFile) {
-    OutputConfigurator.setClientPropertiesFile(CLASS, job.getConfiguration(), clientPropsFile);
-  }
-
   /**
    * Sets the connector information needed to communicate with Accumulo in this job.
    *
@@ -136,11 +106,20 @@ public static void setClientPropertiesFile(Job job, String clientPropsFile) {
    * @param token
    *          the user's password
    * @since 1.5.0
-   * @deprecated since 2.0.0, replaced by {@link #setClientProperties(Job, Properties)}
    */
-  @Deprecated
   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 {
+        org.apache.accumulo.core.client.Instance instance = getInstance(job);
+        org.apache.accumulo.core.client.Connector conn = instance.getConnector(principal, token);
+        token = conn.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) {
@@ -155,7 +134,7 @@ public static void setConnectorInfo(Job job, String principal, AuthenticationTok
       job.getCredentials().addToken(hadoopToken.getService(), hadoopToken);
     }
 
-    OutputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
+    Configurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
   }
 
   /**
@@ -173,12 +152,10 @@ public static void setConnectorInfo(Job job, String principal, AuthenticationTok
    * @param tokenFile
    *          the path to the token file
    * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #setClientPropertiesFile(Job, String)}
    */
-  @Deprecated
   public static void setConnectorInfo(Job job, String principal, String tokenFile)
       throws AccumuloSecurityException {
-    setClientPropertiesFile(job, tokenFile);
+    Configurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
   }
 
   /**
@@ -191,7 +168,7 @@ public static void setConnectorInfo(Job job, String principal, String tokenFile)
    * @see #setConnectorInfo(Job, String, AuthenticationToken)
    */
   protected static Boolean isConnectorInfoSet(JobContext context) {
-    return OutputConfigurator.isConnectorInfoSet(CLASS, context.getConfiguration());
+    return Configurator.isConnectorInfoSet(CLASS, context.getConfiguration());
   }
 
   /**
@@ -204,7 +181,29 @@ protected static Boolean isConnectorInfoSet(JobContext context) {
    * @see #setConnectorInfo(Job, String, AuthenticationToken)
    */
   protected static String getPrincipal(JobContext context) {
-    return OutputConfigurator.getPrincipal(CLASS, context.getConfiguration());
+    return Configurator.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
+  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
+  protected static byte[] getToken(JobContext context) {
+    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
   }
 
   /**
@@ -219,9 +218,29 @@ protected static String getPrincipal(JobContext context) {
    * @see #setConnectorInfo(Job, String, String)
    */
   protected static AuthenticationToken getAuthenticationToken(JobContext context) {
-    AuthenticationToken token = OutputConfigurator.getAuthenticationToken(CLASS,
+    AuthenticationToken token = Configurator.getAuthenticationToken(CLASS,
         context.getConfiguration());
-    return ConfiguratorBase.unwrapAuthenticationToken(context, token);
+    return Configurator.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
+  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
+    setZooKeeperInstance(job, org.apache.accumulo.core.client.ClientConfiguration.create()
+        .withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
@@ -233,12 +252,10 @@ protected static AuthenticationToken getAuthenticationToken(JobContext context)
    * @param clientConfig
    *          client configuration for specifying connection timeouts, SSL connection options, etc.
    * @since 1.6.0
-   * @deprecated since 2.0.0; Use {@link #setClientProperties(Job, Properties)} instead.
    */
-  @Deprecated
   public static void setZooKeeperInstance(Job job,
       org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
+    Configurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
   }
 
   /**
@@ -249,11 +266,9 @@ public static void setZooKeeperInstance(Job job,
    *          the Hadoop context for the configured job
    * @return an Accumulo instance
    * @since 1.5.0
-   * @deprecated since 2.0.0; Use {@link #getClientProperties(JobContext)} instead.
    */
-  @Deprecated
   protected static org.apache.accumulo.core.client.Instance getInstance(JobContext context) {
-    return OutputConfigurator.getInstance(CLASS, context.getConfiguration());
+    return Configurator.getInstance(CLASS, context.getConfiguration());
   }
 
   /**
@@ -266,7 +281,7 @@ public static void setZooKeeperInstance(Job job,
    * @since 1.5.0
    */
   public static void setLogLevel(Job job, Level level) {
-    OutputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
+    Configurator.setLogLevel(CLASS, job.getConfiguration(), level);
   }
 
   /**
@@ -279,7 +294,7 @@ public static void setLogLevel(Job job, Level level) {
    * @see #setLogLevel(Job, Level)
    */
   protected static Level getLogLevel(JobContext context) {
-    return OutputConfigurator.getLogLevel(CLASS, context.getConfiguration());
+    return Configurator.getLogLevel(CLASS, context.getConfiguration());
   }
 
   /**
@@ -293,7 +308,7 @@ protected static Level getLogLevel(JobContext context) {
    * @since 1.5.0
    */
   public static void setDefaultTableName(Job job, String tableName) {
-    OutputConfigurator.setDefaultTableName(CLASS, job.getConfiguration(), tableName);
+    Configurator.setDefaultTableName(CLASS, job.getConfiguration(), tableName);
   }
 
   /**
@@ -306,7 +321,7 @@ public static void setDefaultTableName(Job job, String tableName) {
    * @see #setDefaultTableName(Job, String)
    */
   protected static String getDefaultTableName(JobContext context) {
-    return OutputConfigurator.getDefaultTableName(CLASS, context.getConfiguration());
+    return Configurator.getDefaultTableName(CLASS, context.getConfiguration());
   }
 
   /**
@@ -321,7 +336,7 @@ protected static String getDefaultTableName(JobContext context) {
    * @since 1.5.0
    */
   public static void setBatchWriterOptions(Job job, BatchWriterConfig bwConfig) {
-    OutputConfigurator.setBatchWriterOptions(CLASS, job.getConfiguration(), bwConfig);
+    Configurator.setBatchWriterOptions(CLASS, job.getConfiguration(), bwConfig);
   }
 
   /**
@@ -334,7 +349,7 @@ public static void setBatchWriterOptions(Job job, BatchWriterConfig bwConfig) {
    * @see #setBatchWriterOptions(Job, BatchWriterConfig)
    */
   protected static BatchWriterConfig getBatchWriterOptions(JobContext context) {
-    return OutputConfigurator.getBatchWriterOptions(CLASS, context.getConfiguration());
+    return Configurator.getBatchWriterOptions(CLASS, context.getConfiguration());
   }
 
   /**
@@ -351,7 +366,7 @@ protected static BatchWriterConfig getBatchWriterOptions(JobContext context) {
    * @since 1.5.0
    */
   public static void setCreateTables(Job job, boolean enableFeature) {
-    OutputConfigurator.setCreateTables(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setCreateTables(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
@@ -364,7 +379,7 @@ public static void setCreateTables(Job job, boolean enableFeature) {
    * @see #setCreateTables(Job, boolean)
    */
   protected static Boolean canCreateTables(JobContext context) {
-    return OutputConfigurator.canCreateTables(CLASS, context.getConfiguration());
+    return Configurator.canCreateTables(CLASS, context.getConfiguration());
   }
 
   /**
@@ -381,7 +396,7 @@ protected static Boolean canCreateTables(JobContext context) {
    * @since 1.5.0
    */
   public static void setSimulationMode(Job job, boolean enableFeature) {
-    OutputConfigurator.setSimulationMode(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setSimulationMode(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
@@ -394,7 +409,7 @@ public static void setSimulationMode(Job job, boolean enableFeature) {
    * @see #setSimulationMode(Job, boolean)
    */
   protected static Boolean getSimulationMode(JobContext context) {
-    return OutputConfigurator.getSimulationMode(CLASS, context.getConfiguration());
+    return Configurator.getSimulationMode(CLASS, context.getConfiguration());
   }
 
   /**
@@ -430,7 +445,8 @@ protected AccumuloRecordWriter(TaskAttemptContext context)
       this.defaultTableName = (tname == null) ? null : new Text(tname);
 
       if (!simulate) {
-        this.client = Accumulo.newClient().from(getClientProperties(context)).build();
+        this.client = Accumulo.newClient()
+            .from(Configurator.getClientProperties(CLASS, context.getConfiguration())).build();
         mtbw = client.createMultiTableBatchWriter(getBatchWriterOptions(context));
       }
     }
@@ -569,7 +585,8 @@ public void checkOutputSpecs(JobContext job) throws IOException {
       throw new IOException("Connector info has not been set.");
     String principal = getPrincipal(job);
     AuthenticationToken token = getAuthenticationToken(job);
-    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties(job)).build()) {
+    try (AccumuloClient c = Accumulo.newClient()
+        .from(Configurator.getClientProperties(CLASS, job.getConfiguration())).build()) {
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException | AccumuloSecurityException 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
index 4654f4597c..6cfac2c275 100644
--- 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
@@ -20,6 +20,7 @@
 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;
@@ -40,13 +41,18 @@
  * The user must specify the following via static configurator methods:
  *
  * <ul>
- * <li>{@link AccumuloRowInputFormat#setClientProperties(Job, java.util.Properties)}
+ * <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
 public class AccumuloRowInputFormat
     extends InputFormatBase<Text,PeekingIterator<Entry<Key,Value>>> {
   @Override
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
index 7064188084..76f0400148 100644
--- 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
@@ -19,22 +19,21 @@
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 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.JobConf;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
@@ -54,9 +53,17 @@
  * 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
 public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
 
+  // static wrapper class to make references to deprecated configurator easier
+  private static class Configurator
+      extends org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator {}
+
   /**
    * Gets the table name from the configuration.
    *
@@ -67,7 +74,7 @@
    * @see #setInputTableName(Job, String)
    */
   protected static String getInputTableName(JobContext context) {
-    return InputConfigurator.getInputTableName(CLASS, context.getConfiguration());
+    return Configurator.getInputTableName(CLASS, context.getConfiguration());
   }
 
   /**
@@ -80,7 +87,7 @@ protected static String getInputTableName(JobContext context) {
    * @since 1.5.0
    */
   public static void setInputTableName(Job job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
+    Configurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
   }
 
   /**
@@ -94,7 +101,7 @@ public static void setInputTableName(Job job, String tableName) {
    * @since 1.5.0
    */
   public static void setRanges(Job job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
+    Configurator.setRanges(CLASS, job.getConfiguration(), ranges);
   }
 
   /**
@@ -107,7 +114,7 @@ public static void setRanges(Job job, Collection<Range> ranges) {
    * @see #setRanges(Job, Collection)
    */
   protected static List<Range> getRanges(JobContext context) throws IOException {
-    return InputConfigurator.getRanges(CLASS, context.getConfiguration());
+    return Configurator.getRanges(CLASS, context.getConfiguration());
   }
 
   /**
@@ -123,7 +130,7 @@ public static void setRanges(Job job, Collection<Range> ranges) {
    */
   public static void fetchColumns(Job job,
       Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
+    Configurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
   }
 
   /**
@@ -136,7 +143,7 @@ public static void fetchColumns(Job job,
    * @see #fetchColumns(Job, Collection)
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
-    return InputConfigurator.getFetchedColumns(CLASS, context.getConfiguration());
+    return Configurator.getFetchedColumns(CLASS, context.getConfiguration());
   }
 
   /**
@@ -149,7 +156,7 @@ public static void fetchColumns(Job job,
    * @since 1.5.0
    */
   public static void addIterator(Job job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
+    Configurator.addIterator(CLASS, job.getConfiguration(), cfg);
   }
 
   /**
@@ -163,7 +170,7 @@ public static void addIterator(Job job, IteratorSetting cfg) {
    * @see #addIterator(Job, IteratorSetting)
    */
   protected static List<IteratorSetting> getIterators(JobContext context) {
-    return InputConfigurator.getIterators(CLASS, context.getConfiguration());
+    return Configurator.getIterators(CLASS, context.getConfiguration());
   }
 
   /**
@@ -182,7 +189,7 @@ public static void addIterator(Job job, IteratorSetting cfg) {
    * @since 1.5.0
    */
   public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
-    InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
@@ -196,7 +203,7 @@ public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
    * @see #setAutoAdjustRanges(Job, boolean)
    */
   protected static boolean getAutoAdjustRanges(JobContext context) {
-    return InputConfigurator.getAutoAdjustRanges(CLASS, context.getConfiguration());
+    return Configurator.getAutoAdjustRanges(CLASS, context.getConfiguration());
   }
 
   /**
@@ -212,7 +219,7 @@ protected static boolean getAutoAdjustRanges(JobContext context) {
    * @since 1.5.0
    */
   public static void setScanIsolation(Job job, boolean enableFeature) {
-    InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
@@ -225,7 +232,7 @@ public static void setScanIsolation(Job job, boolean enableFeature) {
    * @see #setScanIsolation(Job, boolean)
    */
   protected static boolean isIsolated(JobContext context) {
-    return InputConfigurator.isIsolated(CLASS, context.getConfiguration());
+    return Configurator.isIsolated(CLASS, context.getConfiguration());
   }
 
   /**
@@ -244,7 +251,7 @@ protected static boolean isIsolated(JobContext context) {
    * @since 1.5.0
    */
   public static void setLocalIterators(Job job, boolean enableFeature) {
-    InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
@@ -257,7 +264,7 @@ public static void setLocalIterators(Job job, boolean enableFeature) {
    * @see #setLocalIterators(Job, boolean)
    */
   protected static boolean usesLocalIterators(JobContext context) {
-    return InputConfigurator.usesLocalIterators(CLASS, context.getConfiguration());
+    return Configurator.usesLocalIterators(CLASS, context.getConfiguration());
   }
 
   /**
@@ -296,7 +303,7 @@ protected static boolean usesLocalIterators(JobContext context) {
    * @since 1.5.0
    */
   public static void setOfflineTableScan(Job job, boolean enableFeature) {
-    InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
@@ -309,14 +316,14 @@ public static void setOfflineTableScan(Job job, boolean enableFeature) {
    * @see #setOfflineTableScan(Job, boolean)
    */
   protected static boolean isOfflineScan(JobContext context) {
-    return InputConfigurator.isOfflineScan(CLASS, context.getConfiguration());
+    return Configurator.isOfflineScan(CLASS, context.getConfiguration());
   }
 
   /**
-   * Controls the use of the {@link org.apache.accumulo.core.client.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)
+   * 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.
@@ -340,12 +347,11 @@ protected static boolean isOfflineScan(JobContext context) {
    * @since 1.7.0
    */
   public static void setBatchScan(Job job, boolean enableFeature) {
-    InputConfigurator.setBatchScan(CLASS, job.getConfiguration(), enableFeature);
+    Configurator.setBatchScan(CLASS, job.getConfiguration(), enableFeature);
   }
 
   /**
-   * Determines whether a configuration has the {@link org.apache.accumulo.core.client.BatchScanner}
-   * feature enabled.
+   * Determines whether a configuration has the {@link BatchScanner} feature enabled.
    *
    * @param context
    *          the Hadoop context for the configured job
@@ -353,7 +359,7 @@ public static void setBatchScan(Job job, boolean enableFeature) {
    * @see #setBatchScan(Job, boolean)
    */
   public static boolean isBatchScan(JobContext context) {
-    return InputConfigurator.isBatchScan(CLASS, context.getConfiguration());
+    return Configurator.isBatchScan(CLASS, context.getConfiguration());
   }
 
   /**
@@ -372,17 +378,7 @@ public static boolean isBatchScan(JobContext context) {
    * @see ScannerBase#setSamplerConfiguration(SamplerConfiguration)
    */
   public static void setSamplerConfiguration(Job job, SamplerConfiguration samplerConfig) {
-    InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
-  }
-
-  /**
-   * Set these execution hints on scanners created for input splits. See
-   * {@link ScannerBase#setExecutionHints(java.util.Map)}
-   *
-   * @since 2.0.0
-   */
-  public static void setExecutionHints(JobConf job, Map<String,String> hints) {
-    InputConfigurator.setExecutionHints(CLASS, job, hints);
+    Configurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
   }
 
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
@@ -391,5 +387,31 @@ public static void setExecutionHints(JobConf job, Map<String,String> hints) {
     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
+    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
+    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
index 9dcfdd29a4..7e31012513 100644
--- 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
@@ -21,13 +21,11 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 
+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;
@@ -40,7 +38,11 @@
 /**
  * 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
 public class InputTableConfig implements Writable {
 
   private List<IteratorSetting> iterators;
@@ -52,7 +54,6 @@
   private boolean useIsolatedScanners = false;
   private boolean offlineScan = false;
   private SamplerConfiguration samplerConfig = null;
-  private Map<String,String> executionHints = Collections.emptyMap();
 
   public InputTableConfig() {}
 
@@ -156,10 +157,10 @@ public boolean shouldAutoAdjustRanges() {
   }
 
   /**
-   * Controls the use of the {@link org.apache.accumulo.core.client.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.
+   * 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>.
@@ -236,7 +237,7 @@ public boolean isOfflineScan() {
   }
 
   /**
-   * Controls the use of the {@link org.apache.accumulo.core.client.IsolatedScanner} in this job.
+   * Controls the use of the {@link IsolatedScanner} in this job.
    *
    * <p>
    * By default, this feature is <b>disabled</b>.
@@ -282,22 +283,6 @@ public SamplerConfiguration getSamplerConfiguration() {
     return samplerConfig;
   }
 
-  /**
-   * The execution hints to set on created scanners. See {@link ScannerBase#setExecutionHints(Map)}
-   *
-   * @since 2.0.0
-   */
-  public void setExecutionHints(Map<String,String> executionHints) {
-    this.executionHints = executionHints;
-  }
-
-  /**
-   * @since 2.0.0
-   */
-  public Map<String,String> getExecutionHints() {
-    return executionHints;
-  }
-
   @Override
   public void write(DataOutput dataOutput) throws IOException {
     if (iterators != null) {
@@ -339,16 +324,6 @@ public void write(DataOutput dataOutput) throws IOException {
       dataOutput.writeBoolean(true);
       new SamplerConfigurationImpl(samplerConfig).write(dataOutput);
     }
-
-    if (executionHints == null || executionHints.size() == 0) {
-      dataOutput.writeInt(0);
-    } else {
-      dataOutput.writeInt(executionHints.size());
-      for (Entry<String,String> entry : executionHints.entrySet()) {
-        dataOutput.writeUTF(entry.getKey());
-        dataOutput.writeUTF(entry.getValue());
-      }
-    }
   }
 
   @Override
@@ -392,14 +367,6 @@ public void readFields(DataInput dataInput) throws IOException {
     if (dataInput.readBoolean()) {
       samplerConfig = new SamplerConfigurationImpl(dataInput).toSamplerConfiguration();
     }
-
-    executionHints = new HashMap<>();
-    int numHints = dataInput.readInt();
-    for (int i = 0; i < numHints; i++) {
-      String k = dataInput.readUTF();
-      String v = dataInput.readUTF();
-      executionHints.put(k, v);
-    }
   }
 
   @Override
@@ -425,11 +392,10 @@ public boolean equals(Object o) {
       return false;
     if (ranges != null ? !ranges.equals(that.ranges) : that.ranges != null)
       return false;
-    if (executionHints != null ? !executionHints.equals(that.executionHints)
-        : that.executionHints != null)
+    if (samplerConfig != null ? !samplerConfig.equals(that.samplerConfig)
+        : that.samplerConfig != null)
       return false;
-    return samplerConfig != null ? samplerConfig.equals(that.samplerConfig)
-        : that.samplerConfig == null;
+    return true;
   }
 
   @Override
@@ -442,7 +408,6 @@ public int hashCode() {
     result = 31 * result + (useIsolatedScanners ? 1 : 0);
     result = 31 * result + (offlineScan ? 1 : 0);
     result = 31 * result + (samplerConfig == null ? 0 : samplerConfig.hashCode());
-    result = 31 * result + (executionHints == null ? 0 : executionHints.hashCode());
     return result;
   }
 }
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
index 867f552ddf..78e462cbdd 100644
--- 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
@@ -16,28 +16,29 @@
  */
 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.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator;
+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;
@@ -46,17 +47,24 @@
 
 /**
  * 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
 public class RangeInputSplit extends InputSplit implements Writable {
   private Range range;
   private String[] locations;
-  private String tableId, tableName;
+  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;
-  private Map<String,String> executionHints;
 
   public RangeInputSplit() {
     range = new Range();
@@ -84,7 +92,8 @@ public Range getRange() {
   }
 
   public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-    return SplitUtils.getProgress(start, end, position);
+    return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getProgress(start, end,
+        position);
   }
 
   public float getProgress(Key currentKey) {
@@ -118,7 +127,7 @@ public float getProgress(Key currentKey) {
    */
   @Override
   public long getLength() throws IOException {
-    return SplitUtils.getRangeLength(range);
+    return org.apache.accumulo.core.clientImpl.mapreduce.SplitUtils.getRangeLength(range);
   }
 
   @Override
@@ -155,7 +164,48 @@ public void readFields(DataInput in) throws IOException {
         columns.add(in.readUTF());
       }
 
-      fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
+      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()) {
@@ -173,14 +223,6 @@ public void readFields(DataInput in) throws IOException {
     if (in.readBoolean()) {
       samplerConfig = new SamplerConfigurationImpl(in).toSamplerConfiguration();
     }
-
-    executionHints = new HashMap<>();
-    int numHints = in.readInt();
-    for (int i = 0; i < numHints; i++) {
-      String k = in.readUTF();
-      String v = in.readUTF();
-      executionHints.put(k, v);
-    }
   }
 
   @Override
@@ -209,13 +251,50 @@ public void write(DataOutput out) throws IOException {
 
     out.writeBoolean(fetchedColumns != null);
     if (fetchedColumns != null) {
-      String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
+      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());
@@ -233,22 +312,32 @@ public void write(DataOutput out) throws IOException {
     if (samplerConfig != null) {
       new SamplerConfigurationImpl(samplerConfig).write(out);
     }
+  }
 
-    if (executionHints == null || executionHints.size() == 0) {
-      out.writeInt(0);
-    } else {
-      out.writeInt(executionHints.size());
-      for (Entry<String,String> entry : executionHints.entrySet()) {
-        out.writeUTF(entry.getKey());
-        out.writeUTF(entry.getValue());
-      }
-    }
+  /**
+   * Use {@link #getTableName}
+   *
+   * @deprecated since 1.6.1, use getTableName() instead.
+   */
+  @Deprecated
+  public String getTable() {
+    return getTableName();
   }
 
   public String getTableName() {
     return tableName;
   }
 
+  /**
+   * Use {@link #setTableName}
+   *
+   * @deprecated since 1.6.1, use setTableName() instead.
+   */
+  @Deprecated
+  public void setTable(String table) {
+    setTableName(table);
+  }
+
   public void setTableName(String table) {
     this.tableName = table;
   }
@@ -261,6 +350,67 @@ public String getTableId() {
     return tableId;
   }
 
+  /**
+   * @see #getInstance(org.apache.accumulo.core.client.ClientConfiguration)
+   * @deprecated since 1.7.0, use getInstance(ClientConfiguration) instead.
+   */
+  @Deprecated
+  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;
   }
@@ -281,6 +431,14 @@ 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;
   }
@@ -331,6 +489,13 @@ public String toString() {
     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);
@@ -338,7 +503,6 @@ public String toString() {
     sb.append(" iterators: ").append(iterators);
     sb.append(" logLevel: ").append(level);
     sb.append(" samplerConfig: ").append(samplerConfig);
-    sb.append(" executionHints: ").append(executionHints);
     return sb.toString();
   }
 
@@ -349,12 +513,4 @@ public void setSamplerConfiguration(SamplerConfiguration samplerConfiguration) {
   public SamplerConfiguration getSamplerConfiguration() {
     return samplerConfig;
   }
-
-  public void setExecutionHints(Map<String,String> executionHints) {
-    this.executionHints = executionHints;
-  }
-
-  public Map<String,String> getExecutionHints() {
-    return executionHints;
-  }
 }
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
index 846c1822d9..1a90ad0948 100644
--- 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
@@ -25,7 +25,11 @@
 
 /**
  * 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
 public class KeyRangePartitioner extends Partitioner<Key,Writable> implements Configurable {
   private RangePartitioner rp = new RangePartitioner();
 
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
index 940bc82d94..365108e907 100644
--- 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
@@ -29,7 +29,6 @@
 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.fs.Path;
@@ -42,7 +41,11 @@
 
 /**
  * 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
 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";
@@ -88,7 +91,8 @@ private synchronized int getNumSubBins() {
   private synchronized Text[] getCutPoints() throws IOException {
     if (cutPointArray == null) {
       String cutFileName = conf.get(CUTFILE_KEY);
-      Path[] cf = DistributedCacheHelper.getLocalCacheFiles(conf);
+      Path[] cf = org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper
+          .getLocalCacheFiles(conf);
 
       if (cf != null) {
         for (Path path : cf) {
@@ -127,7 +131,8 @@ public void setConf(Configuration conf) {
    */
   public static void setSplitFile(Job job, String file) {
     URI uri = new Path(file).toUri();
-    DistributedCacheHelper.addCacheFile(uri, job.getConfiguration());
+    org.apache.accumulo.core.clientImpl.mapreduce.lib.DistributedCacheHelper.addCacheFile(uri,
+        job.getConfiguration());
     job.getConfiguration().set(CUTFILE_KEY, uri.getPath());
   }
 
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
index cda6d453ac..a6eb553bda 100644
--- 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
@@ -28,9 +28,12 @@
 /**
  * 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
 public class BatchInputSplit extends org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit
     implements InputSplit {
 
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
index ebd80f4e2b..6277586c72 100644
--- 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
@@ -24,7 +24,6 @@
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
@@ -33,8 +32,11 @@
 /**
  * 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
  */
-public class BatchInputSplit extends RangeInputSplit {
+@Deprecated
+public class BatchInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit {
   private Collection<Range> ranges;
   private float[] rangeProgress = null;
 
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
index 9e26815ecd..4d687c9a8f 100644
--- 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
@@ -30,7 +30,10 @@
 /**
  * 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
 public class DelegationTokenStub implements AuthenticationToken {
 
   private String serviceName;
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
index d6ecee13e7..887ffaac6e 100644
--- 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
@@ -20,26 +20,27 @@
 import java.io.IOException;
 import java.math.BigInteger;
 
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 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
 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(RangeInputSplit split, InputTableConfig tableConfig,
-      Level logLevel) {
+  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());
-    split.setExecutionHints(tableConfig.getExecutionHints());
   }
 
   public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
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
index c1b954ab18..9b78ecfc65 100644
--- 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
@@ -32,7 +32,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -40,9 +39,7 @@
 import org.apache.accumulo.core.clientImpl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.clientImpl.ClientConfConverter;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
-import org.apache.accumulo.core.clientImpl.ClientInfoImpl;
 import org.apache.accumulo.core.clientImpl.DelegationTokenImpl;
-import org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -59,7 +56,9 @@
 
 /**
  * @since 1.6.0
+ * @deprecated since 2.0.0
  */
+@Deprecated
 public class ConfiguratorBase {
 
   protected static final Logger log = Logger.getLogger(ConfiguratorBase.class);
@@ -70,7 +69,21 @@
    * @since 1.6.0
    */
   public enum ConnectorInfo {
-    IS_CONFIGURED
+    IS_CONFIGURED, PRINCIPAL, TOKEN
+  }
+
+  public static enum TokenSource {
+    FILE, INLINE, JOB;
+
+    private String prefix;
+
+    private TokenSource() {
+      prefix = name().toLowerCase() + ":";
+    }
+
+    public String prefix() {
+      return prefix;
+    }
   }
 
   public enum ClientOpts {
@@ -146,23 +159,6 @@ public static Properties updateToken(org.apache.hadoop.security.Credentials cred
     return result;
   }
 
-  public static ClientInfo getClientInfo(Class<?> implementingClass, Configuration conf) {
-    Properties props = getClientProperties(implementingClass, conf);
-    return new ClientInfoImpl(props);
-  }
-
-  public static void setClientPropertiesFile(Class<?> implementingClass, Configuration conf,
-      String clientPropertiesFile) {
-    try {
-      DistributedCacheHelper.addCacheFile(new URI(clientPropertiesFile), conf);
-    } catch (URISyntaxException e) {
-      throw new IllegalStateException("Unable to add client properties file \""
-          + clientPropertiesFile + "\" to distributed cache.");
-    }
-    conf.set(enumToConfKey(implementingClass, ClientOpts.CLIENT_PROPS_FILE), clientPropertiesFile);
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-  }
-
   public static void setClientProperties(Class<?> implementingClass, Configuration conf,
       Properties props) {
     StringWriter writer = new StringWriter();
@@ -244,6 +240,46 @@ public static void setConnectorInfo(Class<?> implementingClass, Configuration co
     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
   }
 
+  /**
+   * 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) throws AccumuloSecurityException {
+    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");
+
+    try {
+      DistributedCacheHelper.addCacheFile(new URI(tokenFile), conf);
+    } catch (URISyntaxException e) {
+      throw new IllegalStateException(
+          "Unable to add tokenFile \"" + tokenFile + "\" to distributed cache.");
+    }
+
+    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.
    *
@@ -303,10 +339,7 @@ public static AuthenticationToken getAuthenticationToken(Class<?> implementingCl
    * @param clientConfig
    *          client configuration for specifying connection timeouts, SSL connection options, etc.
    * @since 1.6.0
-   * @deprecated since 2.0.0; use {@link #setClientProperties(Class, Configuration, Properties)}
-   *             instead
    */
-  @Deprecated
   public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf,
       org.apache.accumulo.core.client.ClientConfiguration clientConfig) {
     Properties props = getClientProperties(implementingClass, conf);
@@ -329,31 +362,12 @@ public static void setZooKeeperInstance(Class<?> implementingClass, Configuratio
    *          the Hadoop configuration object to configure
    * @return an Accumulo instance
    * @since 1.6.0
-   * @deprecated since 2.0.0, replaced by {@link #getClientInfo(Class, Configuration)}
    */
-  @Deprecated
   public static org.apache.accumulo.core.client.Instance getInstance(Class<?> implementingClass,
       Configuration conf) {
-    try {
-      return org.apache.accumulo.core.client.Connector.from(getClient(implementingClass, conf))
-          .getInstance();
-    } catch (AccumuloSecurityException | AccumuloException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Creates an Accumulo {@link AccumuloClient} based on the configuration
-   *
-   * @param implementingClass
-   *          class whose name will be used as a prefix for the property configuration
-   * @param conf
-   *          Hadoop configuration object
-   * @return Accumulo connector
-   * @since 2.0.0
-   */
-  public static AccumuloClient getClient(Class<?> implementingClass, Configuration conf) {
-    return Accumulo.newClient().from(getClientProperties(implementingClass, conf)).build();
+    ClientInfo info = ClientInfo.from(getClientProperties(implementingClass, conf));
+    return new org.apache.accumulo.core.client.ZooKeeperInstance(info.getZooKeepers(),
+        info.getZooKeepers());
   }
 
   /**
@@ -366,12 +380,10 @@ public static AccumuloClient getClient(Class<?> implementingClass, Configuration
    *
    * @return A ClientConfiguration
    * @since 1.7.0
-   * @deprecated since 2.0.0; use {@link #getClientInfo(Class, Configuration)} instead
    */
-  @Deprecated
   public static org.apache.accumulo.core.client.ClientConfiguration getClientConfiguration(
       Class<?> implementingClass, Configuration conf) {
-    return ClientConfConverter.toClientConf(getClientInfo(implementingClass, conf).getProperties());
+    return ClientConfConverter.toClientConf(getClientProperties(implementingClass, conf));
   }
 
   /**
@@ -433,7 +445,8 @@ public static int getVisibilityCacheSize(Configuration conf) {
 
   /**
    * Unwraps the provided {@link AuthenticationToken} if it is an instance of
-   * {@link DelegationTokenStub}, reconstituting it from the provided {@link JobConf}.
+   * {@link org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub}, reconstituting it
+   * from the provided {@link JobConf}.
    *
    * @param job
    *          The job
@@ -444,8 +457,8 @@ public static AuthenticationToken unwrapAuthenticationToken(JobConf job,
       AuthenticationToken token) {
     requireNonNull(job);
     requireNonNull(token);
-    if (token instanceof DelegationTokenStub) {
-      DelegationTokenStub delTokenStub = (DelegationTokenStub) 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();
@@ -463,7 +476,8 @@ public static AuthenticationToken unwrapAuthenticationToken(JobConf job,
 
   /**
    * Unwraps the provided {@link AuthenticationToken} if it is an instance of
-   * {@link DelegationTokenStub}, reconstituting it from the provided {@link JobConf}.
+   * {@link org.apache.accumulo.core.clientImpl.mapreduce.DelegationTokenStub}, reconstituting it
+   * from the provided {@link JobConf}.
    *
    * @param job
    *          The job
@@ -474,8 +488,8 @@ public static AuthenticationToken unwrapAuthenticationToken(JobContext job,
       AuthenticationToken token) {
     requireNonNull(job);
     requireNonNull(token);
-    if (token instanceof DelegationTokenStub) {
-      DelegationTokenStub delTokenStub = (DelegationTokenStub) 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();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java
index 552b97f1f9..f115ee76ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/DistributedCacheHelper.java
@@ -24,8 +24,9 @@
 
 /**
  * @since 1.6.0
+ * @deprecated since 2.0.0
  */
-@SuppressWarnings("deprecation")
+@Deprecated
 public class DistributedCacheHelper {
 
   /**
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
index f23e880851..105fe5174d 100644
--- 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
@@ -32,7 +32,9 @@
 
 /**
  * @since 1.6.0
+ * @deprecated since 2.0.0
  */
+@Deprecated
 public class FileOutputConfigurator extends ConfiguratorBase {
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
index 58cac6e92a..80c3cf3806 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/InputConfigurator.java
@@ -28,7 +28,6 @@
 import java.util.ArrayList;
 import java.util.Base64;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -37,6 +36,7 @@
 import java.util.Set;
 import java.util.StringTokenizer;
 
+import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -47,13 +47,11 @@
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.clientImpl.Tables;
-import org.apache.accumulo.core.clientImpl.TabletLocator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
@@ -78,7 +76,9 @@
 
 /**
  * @since 1.6.0
+ * @deprecated since 2.0.0
  */
+@Deprecated
 public class InputConfigurator extends ConfiguratorBase {
 
   /**
@@ -634,13 +634,15 @@ public static Boolean isBatchScan(Class<?> implementingClass, Configuration conf
    * @param conf
    *          the Hadoop configuration object to configure
    * @param configs
-   *          an array of {@link InputTableConfig} objects to associate with the job
+   *          an array of {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects
+   *          to associate with the job
    * @since 1.6.0
    */
   public static void setInputTableConfigs(Class<?> implementingClass, Configuration conf,
-      Map<String,InputTableConfig> configs) {
+      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs) {
     MapWritable mapWritable = new MapWritable();
-    for (Map.Entry<String,InputTableConfig> tableConfig : configs.entrySet())
+    for (Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfig : configs
+        .entrySet())
       mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -655,7 +657,8 @@ public static void setInputTableConfigs(Class<?> implementingClass, Configuratio
   }
 
   /**
-   * Returns all {@link InputTableConfig} objects associated with this job.
+   * Returns all {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects
+   * associated with this job.
    *
    * @param implementingClass
    *          the class whose name will be used as a prefix for the property configuration key
@@ -664,11 +667,11 @@ public static void setInputTableConfigs(Class<?> implementingClass, Configuratio
    * @return all of the table query configs for the job
    * @since 1.6.0
    */
-  public static Map<String,InputTableConfig> getInputTableConfigs(Class<?> implementingClass,
-      Configuration conf) {
-    Map<String,InputTableConfig> configs = new HashMap<>();
-    Map.Entry<String,InputTableConfig> defaultConfig = getDefaultInputTableConfig(implementingClass,
-        conf);
+  public static Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> getInputTableConfigs(
+      Class<?> implementingClass, Configuration conf) {
+    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configs = new HashMap<>();
+    Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> defaultConfig = getDefaultInputTableConfig(
+        implementingClass, conf);
     if (defaultConfig != null)
       configs.put(defaultConfig.getKey(), defaultConfig.getValue());
     String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
@@ -685,13 +688,15 @@ public static void setInputTableConfigs(Class<?> implementingClass, Configuratio
       }
     }
     for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet())
-      configs.put(entry.getKey().toString(), (InputTableConfig) entry.getValue());
+      configs.put(entry.getKey().toString(),
+          (org.apache.accumulo.core.client.mapreduce.InputTableConfig) entry.getValue());
 
     return configs;
   }
 
   /**
-   * Returns the {@link InputTableConfig} for the given table
+   * Returns the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the given
+   * table
    *
    * @param implementingClass
    *          the class whose name will be used as a prefix for the property configuration key
@@ -702,31 +707,13 @@ public static void setInputTableConfigs(Class<?> implementingClass, Configuratio
    * @return the table query config for the given table name (if it exists) and null if it does not
    * @since 1.6.0
    */
-  public static InputTableConfig getInputTableConfig(Class<?> implementingClass, Configuration conf,
-      String tableName) {
-    Map<String,InputTableConfig> queryConfigs = getInputTableConfigs(implementingClass, conf);
+  public static org.apache.accumulo.core.client.mapreduce.InputTableConfig getInputTableConfig(
+      Class<?> implementingClass, Configuration conf, String tableName) {
+    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> queryConfigs = getInputTableConfigs(
+        implementingClass, conf);
     return queryConfigs.get(tableName);
   }
 
-  /**
-   * Initializes an Accumulo {@link TabletLocator} 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
-   * @param tableId
-   *          The table id for which to initialize the {@link TabletLocator}
-   * @return an Accumulo tablet locator
-   * @since 1.6.0
-   */
-  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf,
-      Table.ID tableId) {
-    ClientInfo info = getClientInfo(implementingClass, conf);
-    ClientContext context = new ClientContext(info);
-    return TabletLocator.getLocator(context, tableId);
-  }
-
   /**
    * Validates that the user has permissions on the requested tables
    *
@@ -734,29 +721,32 @@ public static TabletLocator getTabletLocator(Class<?> implementingClass, Configu
    *          the class whose name will be used as a prefix for the property configuration key
    * @param conf
    *          the Hadoop configuration object to configure
-   * @param client
-   *          the Accumulo client
    * @since 1.7.0
    */
-  public static void validatePermissions(Class<?> implementingClass, Configuration conf,
-      AccumuloClient client) throws IOException {
-    Map<String,InputTableConfig> inputTableConfigs = getInputTableConfigs(implementingClass, conf);
-    try {
+  public static void validatePermissions(Class<?> implementingClass, Configuration conf)
+      throws IOException {
+    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> inputTableConfigs = getInputTableConfigs(
+        implementingClass, conf);
+    try (AccumuloClient client = Accumulo.newClient()
+        .from(getClientProperties(implementingClass, conf)).build()) {
       if (getInputTableConfigs(implementingClass, conf).size() == 0)
         throw new IOException("No table set.");
 
       String principal = getPrincipal(implementingClass, conf);
       if (principal == null) {
-        principal = getClientInfo(implementingClass, conf).getPrincipal();
+        principal = client.whoami();
       }
 
-      for (Map.Entry<String,InputTableConfig> tableConfig : inputTableConfigs.entrySet()) {
+      for (Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfig : inputTableConfigs
+          .entrySet()) {
         if (!client.securityOperations().hasTablePermission(principal, tableConfig.getKey(),
             TablePermission.READ))
           throw new IOException("Unable to access table");
       }
-      for (Map.Entry<String,InputTableConfig> tableConfigEntry : inputTableConfigs.entrySet()) {
-        InputTableConfig tableConfig = tableConfigEntry.getValue();
+      for (Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> tableConfigEntry : inputTableConfigs
+          .entrySet()) {
+        org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig = tableConfigEntry
+            .getValue();
         if (!tableConfig.shouldUseLocalIterators()) {
           if (tableConfig.getIterators() != null) {
             for (IteratorSetting iter : tableConfig.getIterators()) {
@@ -784,11 +774,11 @@ public static void validatePermissions(Class<?> implementingClass, Configuration
    * @return the config object built from the single input table properties set on the job
    * @since 1.6.0
    */
-  protected static Map.Entry<String,InputTableConfig> getDefaultInputTableConfig(
+  protected static Map.Entry<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> getDefaultInputTableConfig(
       Class<?> implementingClass, Configuration conf) {
     String tableName = getInputTableName(implementingClass, conf);
     if (tableName != null) {
-      InputTableConfig queryConfig = new InputTableConfig();
+      org.apache.accumulo.core.client.mapreduce.InputTableConfig queryConfig = new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
       List<IteratorSetting> itrs = getIterators(implementingClass, conf);
       if (itrs != null)
         queryConfig.setIterators(itrs);
@@ -812,108 +802,110 @@ public static void validatePermissions(Class<?> implementingClass, Configuration
       queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf))
           .setUseIsolatedScanners(isIsolated(implementingClass, conf))
           .setUseLocalIterators(usesLocalIterators(implementingClass, conf))
-          .setOfflineScan(isOfflineScan(implementingClass, conf))
-          .setExecutionHints(getExecutionHints(implementingClass, conf));
+          .setOfflineScan(isOfflineScan(implementingClass, conf));
       return Maps.immutableEntry(tableName, queryConfig);
     }
     return null;
   }
 
   public static Map<String,Map<KeyExtent,List<Range>>> binOffline(Table.ID tableId,
-      List<Range> ranges, ClientContext context)
+      List<Range> ranges, ClientInfo clientInfo)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
 
-    if (Tables.getTableState(context, tableId) != TableState.OFFLINE) {
-      Tables.clearCache(context);
+    try (ClientContext context = new ClientContext(clientInfo)) {
+      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<>();
+
       if (Tables.getTableState(context, tableId) != TableState.OFFLINE) {
-        throw new AccumuloException(
-            "Table is online tableId:" + tableId + " cannot scan table in offline mode ");
+        Tables.clearCache(context);
+        if (Tables.getTableState(context, tableId) != TableState.OFFLINE) {
+          throw new AccumuloException(
+              "Table is online tableId:" + tableId + " cannot scan table in offline mode ");
+        }
       }
-    }
 
-    for (Range range : ranges) {
-      Text startRow;
-
-      if (range.getStartKey() != null)
-        startRow = range.getStartKey().getRow();
-      else
-        startRow = new Text();
-
-      Range metadataRange = new Range(new KeyExtent(tableId, startRow, null).getMetadataEntry(),
-          true, null, false);
-      Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
-      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME);
-      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
-      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME);
-      scanner.setRange(metadataRange);
-
-      RowIterator rowIter = new RowIterator(scanner);
-      KeyExtent lastExtent = null;
-      while (rowIter.hasNext()) {
-        Iterator<Map.Entry<Key,Value>> row = rowIter.next();
-        String last = "";
-        KeyExtent extent = null;
-        String location = null;
-
-        while (row.hasNext()) {
-          Map.Entry<Key,Value> entry = row.next();
-          Key key = entry.getKey();
-
-          if (key.getColumnFamily()
-              .equals(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME)) {
-            last = entry.getValue().toString();
-          }
+      for (Range range : ranges) {
+        Text startRow;
+
+        if (range.getStartKey() != null)
+          startRow = range.getStartKey().getRow();
+        else
+          startRow = new Text();
+
+        Range metadataRange = new Range(new KeyExtent(tableId, startRow, null).getMetadataEntry(),
+            true, null, false);
+        Scanner scanner = context.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+        MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+        scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME);
+        scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
+        scanner.fetchColumnFamily(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME);
+        scanner.setRange(metadataRange);
+
+        RowIterator rowIter = new RowIterator(scanner);
+        KeyExtent lastExtent = null;
+        while (rowIter.hasNext()) {
+          Iterator<Map.Entry<Key,Value>> row = rowIter.next();
+          String last = "";
+          KeyExtent extent = null;
+          String location = null;
+
+          while (row.hasNext()) {
+            Map.Entry<Key,Value> entry = row.next();
+            Key key = entry.getKey();
+
+            if (key.getColumnFamily()
+                .equals(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME)) {
+              last = entry.getValue().toString();
+            }
 
-          if (key.getColumnFamily()
-              .equals(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME)
-              || key.getColumnFamily()
-                  .equals(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME)) {
-            location = entry.getValue().toString();
-          }
+            if (key.getColumnFamily()
+                .equals(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME)
+                || key.getColumnFamily()
+                    .equals(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME)) {
+              location = entry.getValue().toString();
+            }
+
+            if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
+              extent = new KeyExtent(key.getRow(), entry.getValue());
+            }
 
-          if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
-            extent = new KeyExtent(key.getRow(), entry.getValue());
           }
 
-        }
+          if (location != null)
+            return null;
 
-        if (location != null)
-          return null;
+          if (!extent.getTableId().equals(tableId)) {
+            throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
+          }
 
-        if (!extent.getTableId().equals(tableId)) {
-          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
-        }
+          if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
+            throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
+          }
 
-        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
-          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
-        }
+          Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
+          if (tabletRanges == null) {
+            tabletRanges = new HashMap<>();
+            binnedRanges.put(last, tabletRanges);
+          }
 
-        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
-        if (tabletRanges == null) {
-          tabletRanges = new HashMap<>();
-          binnedRanges.put(last, tabletRanges);
-        }
+          List<Range> rangeList = tabletRanges.get(extent);
+          if (rangeList == null) {
+            rangeList = new ArrayList<>();
+            tabletRanges.put(extent, rangeList);
+          }
 
-        List<Range> rangeList = tabletRanges.get(extent);
-        if (rangeList == null) {
-          rangeList = new ArrayList<>();
-          tabletRanges.put(extent, rangeList);
-        }
+          rangeList.add(range);
 
-        rangeList.add(range);
+          if (extent.getEndRow() == null
+              || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
+            break;
+          }
 
-        if (extent.getEndRow() == null
-            || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
-          break;
+          lastExtent = extent;
         }
 
-        lastExtent = extent;
       }
-
+      return binnedRanges;
     }
-    return binnedRanges;
   }
 
   private static String toBase64(Writable writable) {
@@ -950,7 +942,7 @@ public static void setSamplerConfiguration(Class<?> implementingClass, Configura
     conf.set(key, val);
   }
 
-  public static SamplerConfiguration getSamplerConfiguration(Class<?> implementingClass,
+  private static SamplerConfiguration getSamplerConfiguration(Class<?> implementingClass,
       Configuration conf) {
     String key = enumToConfKey(implementingClass, ScanOpts.SAMPLER_CONFIG);
 
@@ -961,30 +953,4 @@ public static SamplerConfiguration getSamplerConfiguration(Class<?> implementing
     return fromBase64(new SamplerConfigurationImpl(), encodedSC).toSamplerConfiguration();
   }
 
-  public static void setExecutionHints(Class<?> implementingClass, Configuration conf,
-      Map<String,String> hints) {
-    MapWritable mapWritable = new MapWritable();
-    hints.forEach((k, v) -> mapWritable.put(new Text(k), new Text(v)));
-
-    String key = enumToConfKey(implementingClass, ScanOpts.EXECUTION_HINTS);
-    String val = toBase64(mapWritable);
-
-    conf.set(key, val);
-  }
-
-  public static Map<String,String> getExecutionHints(Class<?> implementingClass,
-      Configuration conf) {
-    String key = enumToConfKey(implementingClass, ScanOpts.EXECUTION_HINTS);
-    String encodedEH = conf.get(key);
-    if (encodedEH == null) {
-      return Collections.emptyMap();
-    }
-
-    MapWritable mapWritable = new MapWritable();
-    fromBase64(mapWritable, encodedEH);
-
-    HashMap<String,String> hints = new HashMap<>();
-    mapWritable.forEach((k, v) -> hints.put(k.toString(), v.toString()));
-    return hints;
-  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
deleted file mode 100644
index 6848e41751..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnDefaultTable.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-
-import com.beust.jcommander.Parameter;
-
-public class MapReduceClientOnDefaultTable extends MapReduceClientOpts {
-  @Parameter(names = "--table", description = "table to use")
-  public String tableName;
-
-  public MapReduceClientOnDefaultTable(String table) {
-    this.tableName = table;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  @Override
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    super.setAccumuloConfigs(job);
-    final String tableName = getTableName();
-    AccumuloInputFormat.setClientProperties(job, getClientProperties());
-    AccumuloInputFormat.setInputTableName(job, tableName);
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setClientProperties(job, getClientProperties());
-    AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, tableName);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnRequiredTable.java
deleted file mode 100644
index 226638430a..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOnRequiredTable.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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.hadoop.mapreduce.Job;
-
-import com.beust.jcommander.Parameter;
-
-public class MapReduceClientOnRequiredTable extends MapReduceClientOpts {
-
-  @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
-  private String tableName;
-
-  @Override
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    super.setAccumuloConfigs(job);
-    AccumuloInputFormat.setInputTableName(job, getTableName());
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloOutputFormat.setCreateTables(job, true);
-    AccumuloOutputFormat.setDefaultTableName(job, getTableName());
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.java
deleted file mode 100644
index ecf3e3cea1..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/MapReduceClientOpts.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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.accumulo.core.cli.ClientOpts;
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Adds some MR awareness to the ClientOpts
- */
-public class MapReduceClientOpts extends ClientOpts {
-  private static final Logger log = LoggerFactory.getLogger(MapReduceClientOpts.class);
-
-  public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
-    AccumuloInputFormat.setClientProperties(job, getClientProperties());
-    AccumuloOutputFormat.setClientProperties(job, getClientProperties());
-  }
-
-  @Override
-  public AuthenticationToken getToken() {
-    AuthenticationToken authToken = super.getToken();
-    // For MapReduce, Kerberos credentials don't make it to the Mappers and Reducers,
-    // so we need to request a delegation token and use that instead.
-    if (authToken instanceof KerberosToken) {
-      log.info("Received KerberosToken, fetching DelegationToken for MapReduce");
-      final KerberosToken krbToken = (KerberosToken) authToken;
-
-      try {
-        UserGroupInformation user = UserGroupInformation.getCurrentUser();
-        if (!user.hasKerberosCredentials()) {
-          throw new IllegalStateException("Expected current user to have Kerberos credentials");
-        }
-
-        String newPrincipal = user.getUserName();
-        log.info("Obtaining delegation token for {}", newPrincipal);
-
-        setPrincipal(newPrincipal);
-        try (AccumuloClient client = Accumulo.newClient().from(getClientProperties())
-            .as(newPrincipal, krbToken).build()) {
-
-          // Do the explicit check to see if the user has the permission to get a delegation token
-          if (!client.securityOperations().hasSystemPermission(client.whoami(),
-              SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
-            log.error(
-                "{} doesn't have the {} SystemPermission neccesary to obtain a delegation"
-                    + " token. MapReduce tasks cannot automatically use the client's"
-                    + " credentials on remote servers. Delegation tokens provide a means to run"
-                    + " MapReduce without distributing the user's credentials.",
-                user.getUserName(), SystemPermission.OBTAIN_DELEGATION_TOKEN.name());
-            throw new IllegalStateException(
-                client.whoami() + " does not have permission to obtain a delegation token");
-          }
-          // Get the delegation token from Accumulo
-          return client.securityOperations().getDelegationToken(new DelegationTokenConfig());
-        }
-      } catch (Exception e) {
-        final String msg = "Failed to acquire DelegationToken for use with MapReduce";
-        log.error(msg, e);
-        throw new RuntimeException(msg, e);
-      }
-    }
-    return authToken;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java
index 7a3955c7f3..6d003c0285 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/OutputConfigurator.java
@@ -30,7 +30,9 @@
 
 /**
  * @since 1.6.0
+ * @deprecated since 2.0.0
  */
+@Deprecated
 public class OutputConfigurator extends ConfiguratorBase {
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/package-info.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/package-info.java
deleted file mode 100644
index 01d4e4096c..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/package-info.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- * This package exists to store common helpers for configuring MapReduce jobs in a single location.
- * It contains static configurator methods, stored in classes separate from the things they
- * configure (typically, {@link org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat}/
- * {@link org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat} and related classes in
- * compatible frameworks), rather than storing them in those InputFormats/OutputFormats, so as not
- * to clutter their API with methods that don't match the conventions for that framework. These
- * classes may be useful to input/output plugins for other frameworks, so they can reuse the same
- * configuration options and/or serialize them into a {@link org.apache.hadoop.conf.Configuration}
- * instance in a standard way.
- *
- * <p>
- * It is not expected these will change much (except when new features are added), but end users
- * should not use these classes. They should use the static configurators on the
- * {@link org.apache.hadoop.mapreduce.InputFormat} or
- * {@link org.apache.hadoop.mapreduce.OutputFormat} they are configuring, which in turn may use
- * these classes to implement their own static configurators. Once again, these classes are intended
- * for internal use, but may be useful to developers of plugins for other frameworks that read/write
- * to Accumulo.
- *
- * @since 1.6.0
- */
-package org.apache.accumulo.core.clientImpl.mapreduce.lib;
diff --git a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
index 4edd469b91..67242af374 100644
--- a/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
+++ b/core/src/main/java/org/apache/accumulo/core/singletons/SingletonReservation.java
@@ -30,6 +30,12 @@
 
   private static Logger log = LoggerFactory.getLogger(SingletonReservation.class);
 
+  private final Exception e;
+
+  public SingletonReservation() {
+    e = new Exception();
+  }
+
   @Override
   public synchronized void close() {
     if (closed) {
@@ -43,7 +49,7 @@ public synchronized void close() {
   protected void finalize() throws Throwable {
     try {
       if (!closed) {
-        log.warn("An Accumulo Client was garbage collected without being closed.");
+        log.warn("An Accumulo Client was garbage collected without being closed.", e);
       }
     } finally {
       super.finalize();
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
index c5652878e7..41c24dc503 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
@@ -17,24 +17,21 @@
 package org.apache.accumulo.core.client.mapred;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.CountingSummarizer;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
-import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
-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.sample.impl.SamplerConfigurationImpl;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloFileOutputFormatTest {
 
   @Test
@@ -49,11 +46,6 @@ public void validateConfiguration() throws IOException, InterruptedException {
     samplerConfig.addOption("hasher", "murmur3_32");
     samplerConfig.addOption("modulus", "109");
 
-    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(VisibilitySummarizer.class)
-        .addOption(CountingSummarizer.MAX_COUNTERS_OPT, 2048).build();
-    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(FamilySummarizer.class)
-        .addOption(CountingSummarizer.MAX_COUNTERS_OPT, 256).build();
-
     JobConf job = new JobConf();
     AccumuloFileOutputFormat.setReplication(job, a);
     AccumuloFileOutputFormat.setFileBlockSize(job, b);
@@ -61,9 +53,8 @@ public void validateConfiguration() throws IOException, InterruptedException {
     AccumuloFileOutputFormat.setIndexBlockSize(job, d);
     AccumuloFileOutputFormat.setCompressionType(job, e);
     AccumuloFileOutputFormat.setSampler(job, samplerConfig);
-    AccumuloFileOutputFormat.setSummarizers(job, sc1, sc2);
 
-    AccumuloConfiguration acuconf = FileOutputConfigurator
+    AccumuloConfiguration acuconf = org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
         .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
 
     assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
@@ -74,12 +65,6 @@ public void validateConfiguration() throws IOException, InterruptedException {
     assertEquals(new SamplerConfigurationImpl(samplerConfig),
         SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
-    Collection<SummarizerConfiguration> summarizerConfigs = SummarizerConfiguration
-        .fromTableProperties(acuconf);
-    assertEquals(2, summarizerConfigs.size());
-    assertTrue(summarizerConfigs.contains(sc1));
-    assertTrue(summarizerConfigs.contains(sc2));
-
     a = 17;
     b = 1300L;
     c = 150L;
@@ -97,7 +82,8 @@ public void validateConfiguration() throws IOException, InterruptedException {
     AccumuloFileOutputFormat.setCompressionType(job, e);
     AccumuloFileOutputFormat.setSampler(job, samplerConfig);
 
-    acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
+    acuconf = org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
+        .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job);
 
     assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
     assertEquals(1300L, acuconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE));
@@ -107,7 +93,5 @@ public void validateConfiguration() throws IOException, InterruptedException {
     assertEquals(new SamplerConfigurationImpl(samplerConfig),
         SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
-    summarizerConfigs = SummarizerConfiguration.fromTableProperties(acuconf);
-    assertEquals(0, summarizerConfigs.size());
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
index d73d9aef5e..8aa94d4c80 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -33,6 +33,10 @@
 import org.junit.Test;
 import org.junit.rules.TestName;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloInputFormatTest {
 
   private JobConf job;
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
index 1c548ce037..93ac68224a 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
@@ -24,7 +24,6 @@
 import java.util.Map;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
@@ -33,13 +32,18 @@
 import org.junit.Test;
 import org.junit.rules.TestName;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloMultiTableInputFormatTest {
 
   @Rule
   public TestName testName = new TestName();
 
   /**
-   * Verify {@link InputTableConfig} objects get correctly serialized in the JobContext.
+   * Verify {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects get correctly
+   * serialized in the JobContext.
    */
   @Test
   public void testTableQueryConfigSerialization() throws IOException {
@@ -47,17 +51,17 @@ public void testTableQueryConfigSerialization() throws IOException {
     String table2Name = testName.getMethodName() + "2";
     JobConf job = new JobConf();
 
-    InputTableConfig table1 = new InputTableConfig()
+    org.apache.accumulo.core.client.mapreduce.InputTableConfig table1 = new org.apache.accumulo.core.client.mapreduce.InputTableConfig()
         .setRanges(Collections.singletonList(new Range("a", "b")))
         .fetchColumns(Collections.singleton(new Pair<>(new Text("CF1"), new Text("CQ1"))))
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 
-    InputTableConfig table2 = new InputTableConfig()
+    org.apache.accumulo.core.client.mapreduce.InputTableConfig table2 = new org.apache.accumulo.core.client.mapreduce.InputTableConfig()
         .setRanges(Collections.singletonList(new Range("a", "b")))
         .fetchColumns(Collections.singleton(new Pair<>(new Text("CF1"), new Text("CQ1"))))
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 
-    Map<String,InputTableConfig> configMap = new HashMap<>();
+    Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configMap = new HashMap<>();
     configMap.put(table1Name, table1);
     configMap.put(table2Name, table2);
     AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
index c88bce8f18..fd4294215e 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
@@ -27,6 +27,10 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloOutputFormatTest {
 
   @Test
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
index 628f5b630b..0b570c500b 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
@@ -40,8 +40,10 @@
 import org.apache.log4j.Level;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableMap;
-
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class RangeInputSplitTest {
 
   @Test
@@ -89,7 +91,6 @@ public void testAllFieldsWritable() throws IOException {
     split.setFetchedColumns(fetchedColumns);
     split.setIterators(iterators);
     split.setLogLevel(Level.WARN);
-    split.setExecutionHints(ImmutableMap.of("priority", "9"));
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
@@ -110,7 +111,6 @@ public void testAllFieldsWritable() throws IOException {
     assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
     assertEquals(split.getIterators(), newSplit.getIterators());
     assertEquals(split.getLogLevel(), newSplit.getLogLevel());
-    assertEquals(split.getExecutionHints(), newSplit.getExecutionHints());
   }
 
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
index 8ccbb5dbad..4ec28e8c0c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
@@ -17,24 +17,21 @@
 package org.apache.accumulo.core.client.mapreduce;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.client.summary.CountingSummarizer;
-import org.apache.accumulo.core.client.summary.SummarizerConfiguration;
-import org.apache.accumulo.core.client.summary.summarizers.FamilySummarizer;
-import org.apache.accumulo.core.client.summary.summarizers.VisibilitySummarizer;
-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.sample.impl.SamplerConfigurationImpl;
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloFileOutputFormatTest {
 
   @Test
@@ -49,11 +46,6 @@ public void validateConfiguration() throws IOException, InterruptedException {
     samplerConfig.addOption("hasher", "murmur3_32");
     samplerConfig.addOption("modulus", "109");
 
-    SummarizerConfiguration sc1 = SummarizerConfiguration.builder(VisibilitySummarizer.class)
-        .addOption(CountingSummarizer.MAX_COUNTERS_OPT, 2048).build();
-    SummarizerConfiguration sc2 = SummarizerConfiguration.builder(FamilySummarizer.class)
-        .addOption(CountingSummarizer.MAX_COUNTERS_OPT, 256).build();
-
     Job job1 = Job.getInstance();
     AccumuloFileOutputFormat.setReplication(job1, a);
     AccumuloFileOutputFormat.setFileBlockSize(job1, b);
@@ -61,9 +53,8 @@ public void validateConfiguration() throws IOException, InterruptedException {
     AccumuloFileOutputFormat.setIndexBlockSize(job1, d);
     AccumuloFileOutputFormat.setCompressionType(job1, e);
     AccumuloFileOutputFormat.setSampler(job1, samplerConfig);
-    AccumuloFileOutputFormat.setSummarizers(job1, sc1, sc2);
 
-    AccumuloConfiguration acuconf = FileOutputConfigurator
+    AccumuloConfiguration acuconf = org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
         .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job1.getConfiguration());
 
     assertEquals(7, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
@@ -74,12 +65,6 @@ public void validateConfiguration() throws IOException, InterruptedException {
     assertEquals(new SamplerConfigurationImpl(samplerConfig),
         SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
-    Collection<SummarizerConfiguration> summarizerConfigs = SummarizerConfiguration
-        .fromTableProperties(acuconf);
-    assertEquals(2, summarizerConfigs.size());
-    assertTrue(summarizerConfigs.contains(sc1));
-    assertTrue(summarizerConfigs.contains(sc2));
-
     a = 17;
     b = 1300L;
     c = 150L;
@@ -97,8 +82,8 @@ public void validateConfiguration() throws IOException, InterruptedException {
     AccumuloFileOutputFormat.setCompressionType(job2, e);
     AccumuloFileOutputFormat.setSampler(job2, samplerConfig);
 
-    acuconf = FileOutputConfigurator.getAccumuloConfiguration(AccumuloFileOutputFormat.class,
-        job2.getConfiguration());
+    acuconf = org.apache.accumulo.core.clientImpl.mapreduce.lib.FileOutputConfigurator
+        .getAccumuloConfiguration(AccumuloFileOutputFormat.class, job2.getConfiguration());
 
     assertEquals(17, acuconf.getCount(Property.TABLE_FILE_REPLICATION));
     assertEquals(1300L, acuconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE));
@@ -108,8 +93,5 @@ public void validateConfiguration() throws IOException, InterruptedException {
     assertEquals(new SamplerConfigurationImpl(samplerConfig),
         SamplerConfigurationImpl.newSamplerConfig(acuconf));
 
-    summarizerConfigs = SummarizerConfiguration.fromTableProperties(acuconf);
-    assertEquals(0, summarizerConfigs.size());
-
   }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index 0c85bec7c4..314475f05a 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -35,6 +35,10 @@
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloInputFormatTest {
 
   /**
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
index af5667e323..2224f543d1 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@ -32,6 +32,10 @@
 import org.junit.Test;
 import org.junit.rules.TestName;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloMultiTableInputFormatTest {
 
   @Rule
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
index a62474a7d8..232ac0b9da 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
@@ -27,6 +27,10 @@
 import org.apache.hadoop.mapreduce.JobContext;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class AccumuloOutputFormatTest {
 
   @Test
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
index 65519b9968..4b70163962 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
@@ -36,8 +36,10 @@
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableMap;
-
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class InputTableConfigTest {
 
   private InputTableConfig tableQueryConfig;
@@ -113,13 +115,6 @@ public void testSamplerConfig() throws IOException {
     assertEquals(sc, actualConfig.getSamplerConfiguration());
   }
 
-  @Test
-  public void testExecutionHints() throws IOException {
-    tableQueryConfig.setExecutionHints(ImmutableMap.of("priority", "9"));
-    InputTableConfig actualConfig = deserialize(serialize(tableQueryConfig));
-    assertEquals(ImmutableMap.of("priority", "9"), actualConfig.getExecutionHints());
-  }
-
   private byte[] serialize(InputTableConfig tableQueryConfig) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     tableQueryConfig.write(new DataOutputStream(baos));
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
index 18e69b0d74..170c9484dd 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -40,8 +40,10 @@
 import org.apache.log4j.Level;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableMap;
-
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class RangeInputSplitTest {
 
   @Test
@@ -92,7 +94,6 @@ public void testAllFieldsWritable() throws IOException {
     split.setFetchedColumns(fetchedColumns);
     split.setIterators(iterators);
     split.setLogLevel(Level.WARN);
-    split.setExecutionHints(ImmutableMap.of("priority", "9"));
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
@@ -114,7 +115,6 @@ public void testAllFieldsWritable() throws IOException {
     assertEquals(split.getFetchedColumns(), newSplit.getFetchedColumns());
     assertEquals(split.getIterators(), newSplit.getIterators());
     assertEquals(split.getLogLevel(), newSplit.getLogLevel());
-    assertEquals(split.getExecutionHints(), newSplit.getExecutionHints());
   }
 
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
index 60d11477c5..610a450e85 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitionerTest.java
@@ -25,6 +25,10 @@
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class RangePartitionerTest {
 
   private static Text[] cutArray = {new Text("A"), new Text("B"), new Text("C")};
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
index eed2c9e86d..12ee754f24 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/BatchInputSplitTest.java
@@ -43,6 +43,10 @@
 import org.apache.log4j.Level;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class BatchInputSplitTest {
 
   @Test
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
index e1013b21ba..46c4ecdeb9 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/mapreduce/lib/ConfiguratorBaseTest.java
@@ -24,16 +24,20 @@
 import java.util.Properties;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 
+/**
+ * @deprecated since 2.0.0
+ */
+@Deprecated
 public class ConfiguratorBaseTest {
 
   private enum PrivateTestingEnum {
@@ -71,24 +75,12 @@ public void testSetConnectorInfoClassOfQConfigurationStringString()
         new PasswordToken("testPass"));
     assertTrue(ConfiguratorBase.isConnectorInfoSet(this.getClass(), conf));
     assertEquals("testUser", ConfiguratorBase.getPrincipal(this.getClass(), conf));
-    assertEquals("testPass", new String(((PasswordToken) ConfiguratorBase
-        .getClientInfo(this.getClass(), conf).getAuthenticationToken()).getPassword()));
-  }
-
-  @Test
-  public void testSetClientProperties() {
-    Configuration conf = new Configuration();
-    Properties props = Accumulo.newClientProperties().to("myinstance", "myzookeepers")
-        .as("user", "pass").build();
-    ConfiguratorBase.setClientProperties(this.getClass(), conf, props);
-    Properties props2 = ConfiguratorBase.getClientProperties(this.getClass(), conf);
-    assertEquals("myinstance", ClientProperty.INSTANCE_NAME.getValue(props2));
-    assertEquals("myzookeepers", ClientProperty.INSTANCE_ZOOKEEPERS.getValue(props2));
-    assertEquals("user", ClientProperty.AUTH_PRINCIPAL.getValue(props2));
-    assertTrue(ClientProperty.getAuthenticationToken(props2) instanceof PasswordToken);
+    assertEquals("testPass",
+        new String(((PasswordToken) ClientInfo
+            .from(ConfiguratorBase.getClientProperties(this.getClass(), conf))
+            .getAuthenticationToken()).getPassword()));
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void testSetZooKeeperInstance() {
     Configuration conf = new Configuration();
@@ -102,7 +94,7 @@ public void testSetZooKeeperInstance() {
     assertEquals("testInstanceName", clientConf
         .get(org.apache.accumulo.core.client.ClientConfiguration.ClientProperty.INSTANCE_NAME));
 
-    Properties props = ConfiguratorBase.getClientInfo(this.getClass(), conf).getProperties();
+    Properties props = ConfiguratorBase.getClientProperties(this.getClass(), conf);
     assertEquals("testInstanceName", props.getProperty(ClientProperty.INSTANCE_NAME.getKey()));
     assertEquals("testZooKeepers", props.getProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey()));
     assertEquals("true", props.getProperty(ClientProperty.SSL_ENABLED.getKey()));
diff --git a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
index 1a833b1fdb..14369b309e 100644
--- a/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
+++ b/hadoop-mapreduce/src/test/java/org/apache/accumulo/hadoop/its/mapred/AccumuloFileOutputFormatIT.java
@@ -30,19 +30,19 @@
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.rfile.RFileOperations;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.hadoop.mapred.AccumuloFileOutputFormat;
 import org.apache.accumulo.hadoop.mapred.AccumuloInputFormat;
+import org.apache.accumulo.hadoopImpl.mapreduce.lib.ConfiguratorBase;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -193,7 +193,7 @@ private void handleWriteTests(boolean content) throws Exception {
 
       Configuration conf = CachedConfiguration.getInstance();
       DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
-      FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
+      FileSKVIterator sample = FileOperations.getInstance().newReaderBuilder()
           .forFile(files[0].toString(), FileSystem.getLocal(conf), conf,
               CryptoServiceFactory.newDefaultInstance())
           .withTableConfiguration(acuconf).build()
diff --git a/pom.xml b/pom.xml
index 5b07f199da..a856dc9899 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1154,7 +1154,7 @@
                 <module name="OuterTypeFilename" />
                 <module name="LineLength">
                   <property name="max" value="100" />
-                  <property name="ignorePattern" value="^[ ]*[*].*@(link|see) |Map.* = new .*Map|org[.]apache[.]accumulo[.]|a href=|http://|https://|ftp://" />
+                  <property name="ignorePattern" value="^[ ]*[*].*@(link|see) | = new |org[.]apache[.]accumulo[.]|a href=|http://|https://|ftp://" />
                 </module>
                 <module name="AvoidStarImport" />
                 <module name="UnusedImports">
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index ebb2baaa24..22541c5404 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -79,7 +79,7 @@ public ServerContext(SiteConfiguration siteConfig, Properties clientProps) {
     this(siteConfig, ClientInfo.from(clientProps));
   }
 
-  public ServerContext(SiteConfiguration siteConfig, ClientInfo info) {
+  private ServerContext(SiteConfiguration siteConfig, ClientInfo info) {
     this(new ServerInfo(siteConfig, info.getInstanceName(), info.getZooKeepers(),
         info.getZooKeepersSessionTimeOut()));
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
index 7b9d2d9dab..c6f499d64b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CredentialsIT.java
@@ -94,7 +94,7 @@ public void testConnectorWithDestroyedToken() throws Exception {
     assertTrue(token.isDestroyed());
     try (AccumuloClient ignored = Accumulo.newClient().from(getClientInfo().getProperties())
         .as("non_existent_user", token).build()) {
-      fail();
+      fail("should ignore " + ignored);
     } catch (IllegalArgumentException e) {
       assertEquals(e.getMessage(), "AuthenticationToken has been destroyed");
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
index eb231b7122..d07265a0dd 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SslIT.java
@@ -74,10 +74,13 @@ public void bulk() throws Exception {
     }
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void mapReduce() throws Exception {
     try (AccumuloClient client = createClient()) {
-      MapReduceIT.runTest(client, getCluster());
+      // testing old mapreduce code from core jar; the new mapreduce module should have its own test
+      // case which checks functionality with ssl enabled
+      org.apache.accumulo.test.mapreduce.MapReduceIT.runTest(client, getCluster());
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java
index 666897c8d6..05b6db3c3f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZooCacheIT.java
@@ -25,6 +25,7 @@
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.commons.io.FileUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -60,7 +61,7 @@ public void test() throws Exception {
         public void run() {
           try (AccumuloClient client = createClient()) {
             CacheTestReader.main(new String[] {pathName, testDir.getAbsolutePath(),
-                getClientInfo().getZooKeepers()});
+                ClientInfo.from(client.properties()).getZooKeepers()});
           } catch (Exception ex) {
             ref.set(ex);
           }
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
index a69ab45ce1..187ca1109b 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.accumulo.test.mapred;
 
 import static org.junit.Assert.assertEquals;
@@ -29,11 +28,9 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat;
-import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.ConfiguratorBase;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
@@ -62,6 +59,10 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
   private static final Logger log = LoggerFactory.getLogger(AccumuloFileOutputFormatIT.class);
   private static final int JOB_VISIBILITY_CACHE_SIZE = 3000;
@@ -103,7 +104,6 @@ public void testRealWrite() throws Exception {
 
   private static class MRTester extends Configured implements Tool {
     private static class BadKeyMapper implements Mapper<Key,Value,Key,Value> {
-
       int index = 0;
 
       @Override
@@ -150,19 +150,26 @@ public int run(String[] args) throws Exception {
 
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
-      ConfiguratorBase.setVisibilityCacheSize(job, JOB_VISIBILITY_CACHE_SIZE);
-
-      job.setInputFormat(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[1]));
-      AccumuloFileOutputFormat.setSampler(job, SAMPLER_CONFIG);
+      org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator
+          .setVisibilityCacheSize(job, JOB_VISIBILITY_CACHE_SIZE);
+
+      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
+
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table);
+      org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat.setOutputPath(job,
+          new Path(args[1]));
+      org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat.setSampler(job,
+          SAMPLER_CONFIG);
 
       job.setMapperClass(BAD_TABLE.equals(table) ? BadKeyMapper.class : IdentityMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(AccumuloFileOutputFormat.class);
+      job.setOutputFormat(org.apache.accumulo.core.client.mapred.AccumuloFileOutputFormat.class);
 
       job.setNumReduceTasks(0);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
index bee8f0c3fd..3cb14be73c 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
@@ -28,10 +28,9 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapred.RangeInputSplit;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -54,6 +53,10 @@
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloInputFormatIT extends AccumuloClusterHarness {
 
   @BeforeClass
@@ -121,13 +124,18 @@ public int run(String[] args) throws Exception {
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormat(AccumuloInputFormat.class);
+      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
 
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setBatchScan(job, batchScan);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setBatchScan(job, batchScan);
       if (sample) {
-        AccumuloInputFormat.setSamplerConfiguration(job, SAMPLER_CONFIG);
+        org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setSamplerConfiguration(job,
+            SAMPLER_CONFIG);
       }
 
       job.setMapperClass(TestMapper.class);
@@ -219,15 +227,19 @@ public void testCorrectRangeInputSplits() throws Exception {
     try (AccumuloClient accumuloClient = createAccumuloClient()) {
       accumuloClient.tableOperations().create(table);
 
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setScanAuthorizations(job, auths);
-      AccumuloInputFormat.setScanIsolation(job, isolated);
-      AccumuloInputFormat.setLocalIterators(job, localIters);
-      AccumuloInputFormat.fetchColumns(job, fetchColumns);
-      AccumuloInputFormat.setLogLevel(job, level);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setScanAuthorizations(job, auths);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setScanIsolation(job, isolated);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setLocalIterators(job, localIters);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.fetchColumns(job, fetchColumns);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setLogLevel(job, level);
 
-      AccumuloInputFormat aif = new AccumuloInputFormat();
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat aif = new org.apache.accumulo.core.client.mapred.AccumuloInputFormat();
 
       InputSplit[] splits = aif.getSplits(job, 1);
 
@@ -235,9 +247,9 @@ public void testCorrectRangeInputSplits() throws Exception {
 
       InputSplit split = splits[0];
 
-      assertEquals(RangeInputSplit.class, split.getClass());
+      assertEquals(org.apache.accumulo.core.client.mapred.RangeInputSplit.class, split.getClass());
 
-      RangeInputSplit risplit = (RangeInputSplit) split;
+      org.apache.accumulo.core.client.mapred.RangeInputSplit risplit = (org.apache.accumulo.core.client.mapred.RangeInputSplit) split;
 
       assertEquals(table, risplit.getTableName());
       assertEquals(isolated, risplit.isIsolatedScan());
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
index 77fa0e517d..a1ced8c357 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
@@ -27,10 +27,7 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat;
-import org.apache.accumulo.core.client.mapred.RangeInputSplit;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -48,6 +45,10 @@
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloMultiTableInputFormatIT extends AccumuloClusterHarness {
 
   private static AssertionError e1 = null;
@@ -62,7 +63,8 @@
       public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter)
           throws IOException {
         try {
-          String tableName = ((RangeInputSplit) reporter.getInputSplit()).getTableName();
+          String tableName = ((org.apache.accumulo.core.client.mapred.RangeInputSplit) reporter
+              .getInputSplit()).getTableName();
           if (key != null)
             assertEquals(key.getRow().toString(), new String(v.get()));
           assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
@@ -102,18 +104,24 @@ public int run(String[] args) throws Exception {
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormat(AccumuloInputFormat.class);
+      job.setInputFormat(
+          org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.class);
 
-      AccumuloMultiTableInputFormat.setClientProperties(job, getClientProperties());
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
 
-      InputTableConfig tableConfig1 = new InputTableConfig();
-      InputTableConfig tableConfig2 = new InputTableConfig();
+      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig1 = new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
+      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig2 = new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
 
-      Map<String,InputTableConfig> configMap = new HashMap<>();
+      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configMap = new HashMap<>();
       configMap.put(table1, tableConfig1);
       configMap.put(table2, tableConfig2);
 
-      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
+      org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat.setInputTableConfigs(job,
+          configMap);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
index bb43b483f4..3e64643051 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
@@ -27,7 +27,6 @@
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Accumulo;
@@ -36,8 +35,7 @@
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapred.AccumuloOutputFormat;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -59,6 +57,10 @@
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloOutputFormatIT extends ConfigurableMacBase {
 
   @Override
@@ -82,9 +84,14 @@ public void testMapred() throws Exception {
       batchConfig.setMaxWriteThreads(1);
       // set the max memory so that we ensure we don't flush on the write.
       batchConfig.setMaxMemory(Long.MAX_VALUE);
-      AccumuloOutputFormat outputFormat = new AccumuloOutputFormat();
-      AccumuloOutputFormat.setClientProperties(job, getClientProperties());
-      AccumuloOutputFormat.setBatchWriterOptions(job, batchConfig);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat outputFormat = new org.apache.accumulo.core.client.mapred.AccumuloOutputFormat();
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setBatchWriterOptions(job,
+          batchConfig);
       RecordWriter<Text,Mutation> writer = outputFormat.getRecordWriter(null, job, "Test", null);
 
       try {
@@ -167,24 +174,30 @@ public int run(String[] args) throws Exception {
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormat(AccumuloInputFormat.class);
+      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
 
-      Properties clientProps = Accumulo.newClientProperties().to(instanceName, zooKeepers)
-          .as(user, pass).build();
+      ClientInfo info = ClientInfo
+          .from(Accumulo.newClientProperties().to(instanceName, zooKeepers).as(user, pass).build());
 
-      AccumuloInputFormat.setClientProperties(job, clientProps);
-      AccumuloInputFormat.setInputTableName(job, table1);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
+          info.getInstanceName(), info.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job,
+          info.getPrincipal(), info.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table1);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(AccumuloOutputFormat.class);
+      job.setOutputFormat(org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.class);
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.setClientProperties(job, clientProps);
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setZooKeeperInstance(job,
+          info.getInstanceName(), info.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setConnectorInfo(job,
+          info.getPrincipal(), info.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setCreateTables(job, false);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setDefaultTableName(job, table2);
 
       job.setNumReduceTasks(0);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
index 2c2716560c..4e608ea574 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
@@ -31,7 +31,7 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.Mutation;
@@ -53,6 +53,10 @@
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
 
   private static final String ROW1 = "row1";
@@ -157,10 +161,14 @@ public int run(String[] args) throws Exception {
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormat(AccumuloRowInputFormat.class);
+      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.class);
 
-      AccumuloRowInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloRowInputFormat.setInputTableName(job, table);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat.setInputTableName(job, table);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
index 5e3356bad2..326afb2e99 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
@@ -31,8 +31,7 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapred.AccumuloOutputFormat;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -55,6 +54,10 @@
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class TokenFileIT extends AccumuloClusterHarness {
   private static AssertionError e1 = null;
 
@@ -92,7 +95,6 @@ public void close() throws IOException {
 
     }
 
-    @SuppressWarnings("deprecation")
     @Override
     public int run(String[] args) throws Exception {
 
@@ -109,21 +111,28 @@ public int run(String[] args) throws Exception {
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormat(AccumuloInputFormat.class);
+      job.setInputFormat(org.apache.accumulo.core.client.mapred.AccumuloInputFormat.class);
 
-      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloInputFormat.setInputTableName(job, table1);
+      ClientInfo info = getClientInfo();
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setConnectorInfo(job, user,
+          tokenFile);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setInputTableName(job, table1);
+      org.apache.accumulo.core.client.mapred.AccumuloInputFormat.setZooKeeperInstance(job,
+          info.getInstanceName(), info.getZooKeepers());
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(AccumuloOutputFormat.class);
+      job.setOutputFormat(org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.class);
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setConnectorInfo(job, user,
+          tokenFile);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setCreateTables(job, false);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setDefaultTableName(job, table2);
+      org.apache.accumulo.core.client.mapred.AccumuloOutputFormat.setZooKeeperInstance(job,
+          info.getInstanceName(), info.getZooKeepers());
 
       job.setNumReduceTasks(0);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
index b933c0f1ae..6dd9d9baf3 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
@@ -27,10 +27,9 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.data.Key;
@@ -57,6 +56,10 @@
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
 
   private String PREFIX;
@@ -163,18 +166,25 @@ public int run(String[] args) throws Exception {
           this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormatClass(AccumuloInputFormat.class);
+      job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
 
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[1]));
-      AccumuloFileOutputFormat.setSampler(job, SAMPLER_CONFIG);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
+      org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat.setOutputPath(job,
+          new Path(args[1]));
+      org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat.setSampler(job,
+          SAMPLER_CONFIG);
 
       job.setMapperClass(
           table.endsWith("_mapreduce_bad_table") ? BadKeyMapper.class : Mapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloFileOutputFormat.class);
+      job.setOutputFormatClass(
+          org.apache.accumulo.core.client.mapreduce.AccumuloFileOutputFormat.class);
       job.getConfiguration().set("MRTester_tableName", table);
 
       job.setNumReduceTasks(0);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
index 271e50da00..877017f419 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
@@ -38,11 +38,9 @@
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
 import org.apache.accumulo.core.client.sample.RowSampler;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
-import org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -69,9 +67,13 @@
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloInputFormatIT extends AccumuloClusterHarness {
 
-  AccumuloInputFormat inputFormat;
+  org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat inputFormat;
 
   @Override
   protected int defaultTimeoutSeconds() {
@@ -85,7 +87,7 @@ public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoo
 
   @Before
   public void before() {
-    inputFormat = new AccumuloInputFormat();
+    inputFormat = new org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat();
   }
 
   /**
@@ -100,8 +102,12 @@ public void testGetSplits() throws Exception {
       insertData(client, table, currentTimeMillis());
 
       Job job = Job.getInstance();
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
 
       // split table
       TreeSet<Text> splitsToAdd = new TreeSet<>();
@@ -113,20 +119,19 @@ public void testGetSplits() throws Exception {
       // get splits without setting any range
       Collection<Text> actualSplits = client.tableOperations().listSplits(table);
       List<InputSplit> splits = inputFormat.getSplits(job);
-      assertEquals(actualSplits.size() + 1, splits.size()); // No ranges set on the job so it'll
-                                                            // start
-                                                            // with -inf
+      // No ranges set on the job so it'll start with -inf
+      assertEquals(actualSplits.size() + 1, splits.size());
 
       // set ranges and get splits
       List<Range> ranges = new ArrayList<>();
       for (Text text : actualSplits)
         ranges.add(new Range(text));
-      AccumuloInputFormat.setRanges(job, ranges);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setRanges(job, ranges);
       splits = inputFormat.getSplits(job);
       assertEquals(actualSplits.size(), splits.size());
 
       // offline mode
-      AccumuloInputFormat.setOfflineTableScan(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setOfflineTableScan(job, true);
       try {
         inputFormat.getSplits(job);
         fail("An exception should have been thrown");
@@ -141,57 +146,57 @@ public void testGetSplits() throws Exception {
       for (int i = 0; i < 5; i++)
         // overlapping ranges
         ranges.add(new Range(String.format("%09d", i), String.format("%09d", i + 2)));
-      AccumuloInputFormat.setRanges(job, ranges);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setRanges(job, ranges);
       splits = inputFormat.getSplits(job);
       assertEquals(2, splits.size());
 
-      AccumuloInputFormat.setAutoAdjustRanges(job, false);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setAutoAdjustRanges(job, false);
       splits = inputFormat.getSplits(job);
       assertEquals(ranges.size(), splits.size());
 
       // BatchScan not available for offline scans
-      AccumuloInputFormat.setBatchScan(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setBatchScan(job, true);
       // Reset auto-adjust ranges too
-      AccumuloInputFormat.setAutoAdjustRanges(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setAutoAdjustRanges(job, true);
 
-      AccumuloInputFormat.setOfflineTableScan(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setOfflineTableScan(job, true);
       try {
         inputFormat.getSplits(job);
         fail("An exception should have been thrown");
       } catch (IllegalArgumentException e) {}
 
       client.tableOperations().online(table, true);
-      AccumuloInputFormat.setOfflineTableScan(job, false);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setOfflineTableScan(job, false);
 
       // test for resumption of success
       splits = inputFormat.getSplits(job);
       assertEquals(2, splits.size());
 
       // BatchScan not available with isolated iterators
-      AccumuloInputFormat.setScanIsolation(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, true);
       try {
         inputFormat.getSplits(job);
         fail("An exception should have been thrown");
       } catch (IllegalArgumentException e) {}
-      AccumuloInputFormat.setScanIsolation(job, false);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, false);
 
       // test for resumption of success
       splits = inputFormat.getSplits(job);
       assertEquals(2, splits.size());
 
       // BatchScan not available with local iterators
-      AccumuloInputFormat.setLocalIterators(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job, true);
       try {
         inputFormat.getSplits(job);
         fail("An exception should have been thrown");
       } catch (IllegalArgumentException e) {}
-      AccumuloInputFormat.setLocalIterators(job, false);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job, false);
 
       // Check we are getting back correct type pf split
       client.tableOperations().online(table);
       splits = inputFormat.getSplits(job);
       for (InputSplit split : splits)
-        assert (split instanceof BatchInputSplit);
+        assert (split instanceof org.apache.accumulo.core.clientImpl.mapreduce.BatchInputSplit);
 
       // We should divide along the tablet lines similar to when using `setAutoAdjustRanges(job,
       // true)`
@@ -283,11 +288,16 @@ public int run(String[] args) throws Exception {
 
       job.setInputFormatClass(inputFormatClass);
 
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setBatchScan(job, batchScan);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setBatchScan(job, batchScan);
       if (sample) {
-        AccumuloInputFormat.setSamplerConfiguration(job, SAMPLER_CONFIG);
+        org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setSamplerConfiguration(job,
+            SAMPLER_CONFIG);
       }
 
       job.setMapperClass(TestMapper.class);
@@ -325,8 +335,8 @@ public void testMap() throws Exception {
       }
       bw.close();
 
-      assertEquals(0,
-          MRTester.main(new String[] {TEST_TABLE_1, AccumuloInputFormat.class.getName()}));
+      assertEquals(0, MRTester.main(new String[] {TEST_TABLE_1,
+          org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName()}));
       assertEquals(1, assertionErrors.get(TEST_TABLE_1 + "_map").size());
       assertEquals(1, assertionErrors.get(TEST_TABLE_1 + "_cleanup").size());
     }
@@ -350,20 +360,26 @@ public void testSample() throws Exception {
       }
       bw.close();
 
-      assertEquals(0, MRTester
-          .main(new String[] {TEST_TABLE_3, AccumuloInputFormat.class.getName(), "False", "True"}));
+      assertEquals(0,
+          MRTester.main(new String[] {TEST_TABLE_3,
+              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(),
+              "False", "True"}));
       assertEquals(39, assertionErrors.get(TEST_TABLE_3 + "_map").size());
       assertEquals(2, assertionErrors.get(TEST_TABLE_3 + "_cleanup").size());
 
       assertionErrors.clear();
-      assertEquals(0, MRTester.main(
-          new String[] {TEST_TABLE_3, AccumuloInputFormat.class.getName(), "False", "False"}));
+      assertEquals(0,
+          MRTester.main(new String[] {TEST_TABLE_3,
+              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(),
+              "False", "False"}));
       assertEquals(1, assertionErrors.get(TEST_TABLE_3 + "_map").size());
       assertEquals(1, assertionErrors.get(TEST_TABLE_3 + "_cleanup").size());
 
       assertionErrors.clear();
-      assertEquals(0, MRTester
-          .main(new String[] {TEST_TABLE_3, AccumuloInputFormat.class.getName(), "True", "True"}));
+      assertEquals(0,
+          MRTester.main(new String[] {TEST_TABLE_3,
+              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(), "True",
+              "True"}));
       assertEquals(39, assertionErrors.get(TEST_TABLE_3 + "_map").size());
       assertEquals(2, assertionErrors.get(TEST_TABLE_3 + "_cleanup").size());
     }
@@ -383,8 +399,10 @@ public void testMapWithBatchScanner() throws Exception {
       }
       bw.close();
 
-      assertEquals(0, MRTester
-          .main(new String[] {TEST_TABLE_2, AccumuloInputFormat.class.getName(), "True", "False"}));
+      assertEquals(0,
+          MRTester.main(new String[] {TEST_TABLE_2,
+              org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class.getName(), "True",
+              "False"}));
       assertEquals(1, assertionErrors.get(TEST_TABLE_2 + "_map").size());
       assertEquals(1, assertionErrors.get(TEST_TABLE_2 + "_cleanup").size());
     }
@@ -404,15 +422,21 @@ public void testCorrectRangeInputSplits() throws Exception {
     try (AccumuloClient accumuloClient = createAccumuloClient()) {
       accumuloClient.tableOperations().create(table);
 
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setScanAuthorizations(job, auths);
-      AccumuloInputFormat.setScanIsolation(job, isolated);
-      AccumuloInputFormat.setLocalIterators(job, localIters);
-      AccumuloInputFormat.fetchColumns(job, fetchColumns);
-      AccumuloInputFormat.setLogLevel(job, level);
-
-      AccumuloInputFormat aif = new AccumuloInputFormat();
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanAuthorizations(job,
+          auths);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanIsolation(job, isolated);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLocalIterators(job,
+          localIters);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.fetchColumns(job, fetchColumns);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setLogLevel(job, level);
+
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat aif = new org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat();
 
       List<InputSplit> splits = aif.getSplits(job);
 
@@ -420,9 +444,10 @@ public void testCorrectRangeInputSplits() throws Exception {
 
       InputSplit split = splits.get(0);
 
-      assertEquals(RangeInputSplit.class, split.getClass());
+      assertEquals(org.apache.accumulo.core.client.mapreduce.RangeInputSplit.class,
+          split.getClass());
 
-      RangeInputSplit risplit = (RangeInputSplit) split;
+      org.apache.accumulo.core.client.mapreduce.RangeInputSplit risplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
 
       assertEquals(table, risplit.getTableName());
       assertEquals(isolated, risplit.isIsolatedScan());
@@ -455,7 +480,8 @@ public void testPartialInputSplitDelegationToConfiguration() throws Exception {
   /**
    * AccumuloInputFormat which returns an "empty" RangeInputSplit
    */
-  public static class EmptySplitsAccumuloInputFormat extends AccumuloInputFormat {
+  public static class EmptySplitsAccumuloInputFormat
+      extends org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat {
 
     @Override
     public List<InputSplit> getSplits(JobContext context) throws IOException {
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
index b996a70240..2d20e495a1 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
@@ -27,9 +27,7 @@
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -44,6 +42,10 @@
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloMultiTableInputFormatIT extends AccumuloClusterHarness {
 
   private static AssertionError e1 = null;
@@ -58,7 +60,8 @@
       @Override
       protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
         try {
-          String tableName = ((RangeInputSplit) context.getInputSplit()).getTableName();
+          String tableName = ((org.apache.accumulo.core.client.mapreduce.RangeInputSplit) context
+              .getInputSplit()).getTableName();
           if (key != null)
             assertEquals(key.getRow().toString(), new String(v.get()));
           assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
@@ -95,18 +98,24 @@ public int run(String[] args) throws Exception {
           this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormatClass(AccumuloMultiTableInputFormat.class);
+      job.setInputFormatClass(
+          org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat.class);
 
-      AccumuloMultiTableInputFormat.setClientProperties(job, getClientProperties());
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat
+          .setZooKeeperInstance(job, ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
 
-      InputTableConfig tableConfig1 = new InputTableConfig();
-      InputTableConfig tableConfig2 = new InputTableConfig();
+      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig1 = new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
+      org.apache.accumulo.core.client.mapreduce.InputTableConfig tableConfig2 = new org.apache.accumulo.core.client.mapreduce.InputTableConfig();
 
-      Map<String,InputTableConfig> configMap = new HashMap<>();
+      Map<String,org.apache.accumulo.core.client.mapreduce.InputTableConfig> configMap = new HashMap<>();
       configMap.put(table1, tableConfig1);
       configMap.put(table2, tableConfig2);
 
-      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
+      org.apache.accumulo.core.client.mapreduce.AccumuloMultiTableInputFormat
+          .setInputTableConfigs(job, configMap);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
index d135b2b524..ebc954af77 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
@@ -30,8 +30,7 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -46,6 +45,10 @@
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
   private static AssertionError e1 = null;
 
@@ -91,21 +94,30 @@ public int run(String[] args) throws Exception {
           this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormatClass(AccumuloInputFormat.class);
+      job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
 
-      AccumuloInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table1);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table1);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloOutputFormat.class);
+      job.setOutputFormatClass(
+          org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class);
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.setClientProperties(job, getClientProperties());
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setCreateTables(job, false);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setDefaultTableName(job,
+          table2);
 
       job.setNumReduceTasks(0);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
index 5d7c08d2b3..3849a913dd 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
@@ -31,8 +31,7 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.Mutation;
@@ -51,6 +50,10 @@
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
 
   private static final String ROW1 = "row1";
@@ -152,10 +155,16 @@ public int run(String[] args) throws Exception {
           this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormatClass(AccumuloRowInputFormat.class);
+      job.setInputFormatClass(
+          org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.class);
 
-      AccumuloRowInputFormat.setClientProperties(job, getClientProperties());
-      AccumuloInputFormat.setInputTableName(job, table);
+      ClientInfo ci = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.setZooKeeperInstance(job,
+          ci.getInstanceName(), ci.getZooKeepers());
+      org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.setConnectorInfo(job,
+          ci.getPrincipal(), ci.getAuthenticationToken());
+      org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat.setInputTableName(job,
+          table);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
similarity index 89%
rename from test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java
rename to test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
index dc16e07bc1..33b13c252a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MapReduceIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/MapReduceIT.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.test.functional;
+package org.apache.accumulo.test.mapreduce;
 
 import static org.junit.Assert.assertEquals;
 
@@ -40,12 +40,16 @@
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.test.mapreduce.RowHash;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class MapReduceIT extends ConfigurableMacBase {
 
   @Override
@@ -71,9 +75,10 @@ public void test() throws Exception {
   }
 
   @SuppressFBWarnings(value = "WEAK_MESSAGE_DIGEST_MD5", justification = "md5 is okay for testing")
-  static void runTest(AccumuloClient c, MiniAccumuloClusterImpl cluster) throws AccumuloException,
-      AccumuloSecurityException, TableExistsException, TableNotFoundException,
-      MutationsRejectedException, IOException, InterruptedException, NoSuchAlgorithmException {
+  public static void runTest(AccumuloClient c, MiniAccumuloClusterImpl cluster)
+      throws AccumuloException, AccumuloSecurityException, TableExistsException,
+      TableNotFoundException, MutationsRejectedException, IOException, InterruptedException,
+      NoSuchAlgorithmException {
     c.tableOperations().create(tablename);
     BatchWriter bw = c.createBatchWriter(tablename, new BatchWriterConfig());
     for (int i = 0; i < 10; i++) {
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.java
index 3ea748f03e..a11218bc22 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/RowHash.java
@@ -20,12 +20,17 @@
 import java.util.Base64;
 import java.util.Collections;
 
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.clientImpl.mapreduce.lib.MapReduceClientOnRequiredTable;
+import org.apache.accumulo.core.cli.ClientOpts;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.admin.DelegationTokenConfig;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -33,11 +38,18 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.beust.jcommander.Parameter;
 
+/**
+ * This class supports deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class RowHash extends Configured implements Tool {
   /**
    * The Mapper class that given a row number, will generate the appropriate output line.
@@ -56,9 +68,81 @@ public void map(Key row, Value data, Context context) throws IOException, Interr
     public void setup(Context job) {}
   }
 
-  private static class Opts extends MapReduceClientOnRequiredTable {
+  private static class Opts extends ClientOpts {
+    private static final Logger log = LoggerFactory.getLogger(Opts.class);
+
     @Parameter(names = "--column", required = true)
     String column;
+
+    @Parameter(names = {"-t", "--table"}, required = true, description = "table to use")
+    private String tableName;
+
+    public String getTableName() {
+      return tableName;
+    }
+
+    public void setAccumuloConfigs(Job job) throws AccumuloSecurityException {
+      org.apache.accumulo.core.clientImpl.mapreduce.lib.InputConfigurator.setClientProperties(
+          org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class,
+          job.getConfiguration(), this.getClientProperties());
+      org.apache.accumulo.core.clientImpl.mapreduce.lib.OutputConfigurator.setClientProperties(
+          org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class,
+          job.getConfiguration(), this.getClientProperties());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job,
+          getTableName());
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setScanAuthorizations(job,
+          auths);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setCreateTables(job, true);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setDefaultTableName(job,
+          getTableName());
+    }
+
+    @Override
+    public AuthenticationToken getToken() {
+      AuthenticationToken authToken = super.getToken();
+      // For MapReduce, Kerberos credentials don't make it to the Mappers and Reducers,
+      // so we need to request a delegation token and use that instead.
+      if (authToken instanceof KerberosToken) {
+        log.info("Received KerberosToken, fetching DelegationToken for MapReduce");
+        final KerberosToken krbToken = (KerberosToken) authToken;
+
+        try {
+          UserGroupInformation user = UserGroupInformation.getCurrentUser();
+          if (!user.hasKerberosCredentials()) {
+            throw new IllegalStateException("Expected current user to have Kerberos credentials");
+          }
+
+          String newPrincipal = user.getUserName();
+          log.info("Obtaining delegation token for {}", newPrincipal);
+
+          setPrincipal(newPrincipal);
+          AccumuloClient client = Accumulo.newClient().from(getClientProperties())
+              .as(newPrincipal, krbToken).build();
+
+          // Do the explicit check to see if the user has the permission to get a delegation token
+          if (!client.securityOperations().hasSystemPermission(client.whoami(),
+              SystemPermission.OBTAIN_DELEGATION_TOKEN)) {
+            log.error(
+                "{} doesn't have the {} SystemPermission neccesary to obtain a delegation"
+                    + " token. MapReduce tasks cannot automatically use the client's"
+                    + " credentials on remote servers. Delegation tokens provide a means to run"
+                    + " MapReduce without distributing the user's credentials.",
+                user.getUserName(), SystemPermission.OBTAIN_DELEGATION_TOKEN.name());
+            throw new IllegalStateException(
+                client.whoami() + " does not have permission to obtain a delegation token");
+          }
+
+          // Get the delegation token from Accumulo
+          return client.securityOperations().getDelegationToken(new DelegationTokenConfig());
+        } catch (Exception e) {
+          final String msg = "Failed to acquire DelegationToken for use with MapReduce";
+          log.error(msg, e);
+          throw new RuntimeException(msg, e);
+        }
+      }
+      return authToken;
+    }
+
   }
 
   @Override
@@ -68,7 +152,7 @@ public int run(String[] args) throws Exception {
     job.setJarByClass(this.getClass());
     Opts opts = new Opts();
     opts.parseArgs(RowHash.class.getName(), args);
-    job.setInputFormatClass(AccumuloInputFormat.class);
+    job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
     opts.setAccumuloConfigs(job);
 
     String col = opts.column;
@@ -76,7 +160,8 @@ public int run(String[] args) throws Exception {
     Text cf = new Text(idx < 0 ? col : col.substring(0, idx));
     Text cq = idx < 0 ? null : new Text(col.substring(idx + 1));
     if (cf.getLength() > 0)
-      AccumuloInputFormat.fetchColumns(job, Collections.singleton(new Pair<>(cf, cq)));
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.fetchColumns(job,
+          Collections.singleton(new Pair<>(cf, cq)));
 
     job.setMapperClass(HashDataMapper.class);
     job.setMapOutputKeyClass(Text.class);
@@ -84,7 +169,7 @@ public int run(String[] args) throws Exception {
 
     job.setNumReduceTasks(0);
 
-    job.setOutputFormatClass(AccumuloOutputFormat.class);
+    job.setOutputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class);
 
     job.waitForCompletion(true);
     return job.isSuccessful() ? 0 : 1;
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
index ee65c690e4..2baebf01a3 100644
--- a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
@@ -31,8 +31,7 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.clientImpl.ClientInfo;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -52,6 +51,10 @@
 
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
+/**
+ * This tests deprecated mapreduce code in core jar
+ */
+@Deprecated
 public class TokenFileIT extends AccumuloClusterHarness {
   private static AssertionError e1 = null;
 
@@ -82,7 +85,6 @@ protected void cleanup(Context context) throws IOException, InterruptedException
       }
     }
 
-    @SuppressWarnings("deprecation")
     @Override
     public int run(String[] args) throws Exception {
 
@@ -100,21 +102,30 @@ public int run(String[] args) throws Exception {
           this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
       job.setJarByClass(this.getClass());
 
-      job.setInputFormatClass(AccumuloInputFormat.class);
+      job.setInputFormatClass(org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.class);
 
-      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloInputFormat.setInputTableName(job, table1);
+      ClientInfo info = getClientInfo();
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setConnectorInfo(job, user,
+          tokenFile);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setInputTableName(job, table1);
+      org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat.setZooKeeperInstance(job,
+          info.getInstanceName(), info.getZooKeepers());
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloOutputFormat.class);
+      job.setOutputFormatClass(
+          org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.class);
       job.setOutputKeyClass(Text.class);
       job.setOutputValueClass(Mutation.class);
 
-      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setConnectorInfo(job, user,
+          tokenFile);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setCreateTables(job, false);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setDefaultTableName(job,
+          table2);
+      org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat.setZooKeeperInstance(job,
+          info.getInstanceName(), info.getZooKeepers());
 
       job.setNumReduceTasks(0);
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services