You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2016/06/30 20:41:50 UTC

[7/8] accumulo git commit: ACCUMULO-2589 Drop deprecated code from 1.x

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
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 0cf57d2..2523819 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
@@ -24,10 +24,7 @@ import java.util.Set;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Key;
@@ -355,22 +352,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
     InputConfigurator.setSamplerConfiguration(CLASS, job, samplerConfig);
   }
 
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param job
-   *          the Hadoop job for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the job doesn't exist
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobConf job) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, job, InputConfigurator.getInputTableName(CLASS, job));
-  }
-
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
 
     @Override
@@ -378,56 +359,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
       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);
-    }
   }
 
-  /**
-   * @deprecated since 1.5.2; Use {@link org.apache.accumulo.core.client.mapred.RangeInputSplit} instead.
-   * @see org.apache.accumulo.core.client.mapred.RangeInputSplit
-   */
-  @Deprecated
-  public static class RangeInputSplit extends org.apache.accumulo.core.client.mapred.RangeInputSplit {
-    public RangeInputSplit() {
-      super();
-    }
-
-    public RangeInputSplit(RangeInputSplit other) throws IOException {
-      super(other);
-    }
-
-    public RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-      super(table, tableId, range, locations);
-    }
-
-    protected RangeInputSplit(String table, Range range, String[] locations) {
-      super(table, "", range, locations);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
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 e7cb8ec..abf8538 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
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.core.client.mapreduce;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -67,7 +69,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -81,8 +82,6 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-
 /**
  * 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}.
@@ -206,28 +205,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
   }
 
   /**
-   * 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));
-  }
-
-  /**
    * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
    *
    * @param context
@@ -247,23 +224,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
    *
    * @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, ClientConfiguration)} instead.
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
    *
    * @param clientConfig
    *          client configuration containing connection options
@@ -274,21 +234,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
   }
 
   /**
-   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param instanceName
-   *          the Accumulo instance name
-   * @since 1.5.0
-   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
-   */
-  @Deprecated
-  public static void setMockInstance(Job job, String instanceName) {
-    InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
-  }
-
-  /**
    * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the configuration.
    *
    * @param context
@@ -381,25 +326,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     return InputConfigurator.getInputTableConfig(CLASS, context.getConfiguration(), tableName);
   }
 
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @param table
-   *          the table for which to initialize the locator
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.6.0
-   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-   *             discourage its use.
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), table);
-  }
-
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
   /**
    * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
@@ -498,23 +424,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
         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 {
 
@@ -591,8 +500,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
         try {
           if (isOffline) {
             scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
-          } else if (DeprecationUtil.isMockInstance(instance)) {
-            scanner = instance.getConnector(principal, token).createScanner(split.getTableName(), authorizations);
           } else {
             ClientConfiguration clientConf = getClientConfiguration(attempt);
             ClientContext context = new ClientContext(instance, new Credentials(principal, token), clientConf);
@@ -718,14 +625,10 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       Instance instance = getInstance(context);
       String tableId;
       // resolve table name to id once, and use id from this point forward
-      if (DeprecationUtil.isMockInstance(instance)) {
-        tableId = "";
-      } else {
-        try {
-          tableId = Tables.getTableId(instance, tableName);
-        } catch (TableNotFoundException e) {
-          throw new IOException(e);
-        }
+      try {
+        tableId = Tables.getTableId(instance, tableName);
+      } catch (TableNotFoundException e) {
+        throw new IOException(e);
       }
 
       Authorizations auths = getScanAuthorizations(context);
@@ -768,12 +671,10 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
           ClientContext clientContext = new ClientContext(getInstance(context), new Credentials(getPrincipal(context), getAuthenticationToken(context)),
               getClientConfiguration(context));
           while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
-            if (!DeprecationUtil.isMockInstance(instance)) {
-              if (!Tables.exists(instance, tableId))
-                throw new TableDeletedException(tableId);
-              if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-                throw new TableOfflineException(instance, tableId);
-            }
+            if (!Tables.exists(instance, tableId))
+              throw new TableDeletedException(tableId);
+            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+              throw new TableOfflineException(instance, tableId);
             binnedRanges.clear();
             log.warn("Unable to locate bins for specified ranges. Retrying.");
             // sleep randomly between 100 and 200 ms

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
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 75afe2b..656dba7 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
@@ -30,7 +30,6 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
@@ -52,21 +51,6 @@ public class AccumuloFileOutputFormat extends FileOutputFormat<Key,Value> {
   protected static final Logger log = Logger.getLogger(CLASS);
 
   /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @since 1.5.0
-   * @deprecated since 1.7.0 This method returns a type that is not part of the public API and is not guaranteed to be stable. The method was deprecated to
-   *             discourage its use.
-   */
-  @Deprecated
-  protected static AccumuloConfiguration getAccumuloConfiguration(JobContext context) {
-    return FileOutputConfigurator.getAccumuloConfiguration(CLASS, context.getConfiguration());
-  }
-
-  /**
    * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
    *
    * @param job

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
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 fa926b6..40e4ce7 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
@@ -42,7 +42,6 @@ import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
 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;
@@ -170,28 +169,6 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
   }
 
   /**
-   * 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));
-  }
-
-  /**
    * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
    *
    * @param context
@@ -211,23 +188,6 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    *
    * @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, ClientConfiguration)} instead.
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    setZooKeeperInstance(job, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link ZooKeeperInstance} for this job.
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
    *
    * @param clientConfig
    *          client configuration for specifying connection timeouts, SSL connection options, etc.
@@ -238,20 +198,6 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
   }
 
   /**
-   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
-   *
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param instanceName
-   *          the Accumulo instance name
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setMockInstance(Job job, String instanceName) {
-    OutputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
-  }
-
-  /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    *
    * @param context

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
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 324d5c7..c6ae5a2 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
@@ -24,10 +24,7 @@ import java.util.Set;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Key;
@@ -354,22 +351,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
     InputConfigurator.setSamplerConfiguration(CLASS, job.getConfiguration(), samplerConfig);
   }
 
-  /**
-   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
-   *
-   * @param context
-   *          the Hadoop context for the configured job
-   * @return an Accumulo tablet locator
-   * @throws org.apache.accumulo.core.client.TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.5.0
-   * @deprecated since 1.6.0
-   */
-  @Deprecated
-  protected static TabletLocator getTabletLocator(JobContext context) throws TableNotFoundException {
-    return InputConfigurator.getTabletLocator(CLASS, context.getConfiguration(), InputConfigurator.getInputTableName(CLASS, context.getConfiguration()));
-  }
-
   protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
 
     @Override
@@ -377,53 +358,6 @@ public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
       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);
-    }
   }
 
-  /**
-   * @deprecated since 1.5.2; Use {@link org.apache.accumulo.core.client.mapreduce.RangeInputSplit} instead.
-   * @see org.apache.accumulo.core.client.mapreduce.RangeInputSplit
-   */
-  @Deprecated
-  public static class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.RangeInputSplit {
-
-    public RangeInputSplit() {
-      super();
-    }
-
-    public RangeInputSplit(RangeInputSplit other) throws IOException {
-      super(other);
-    }
-
-    protected RangeInputSplit(String table, Range range, String[] locations) {
-      super(table, "", range, locations);
-    }
-
-    public RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-      super(table, tableId, range, locations);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
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 3849432..63691f4 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
@@ -45,7 +45,6 @@ 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.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -62,7 +61,7 @@ public class RangeInputSplit extends InputSplit implements Writable {
   private TokenSource tokenSource;
   private String tokenFile;
   private AuthenticationToken token;
-  private Boolean offline, mockInstance, isolatedScan, localIterators;
+  private Boolean offline, isolatedScan, localIterators;
   private Authorizations auths;
   private Set<Pair<Text,Text>> fetchedColumns;
   private List<IteratorSetting> iterators;
@@ -155,10 +154,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     }
 
     if (in.readBoolean()) {
-      mockInstance = in.readBoolean();
-    }
-
-    if (in.readBoolean()) {
       int numColumns = in.readInt();
       List<String> columns = new ArrayList<String>(numColumns);
       for (int i = 0; i < numColumns; i++) {
@@ -247,11 +242,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
       out.writeBoolean(localIterators);
     }
 
-    out.writeBoolean(null != mockInstance);
-    if (null != mockInstance) {
-      out.writeBoolean(mockInstance);
-    }
-
     out.writeBoolean(null != fetchedColumns);
     if (null != fetchedColumns) {
       String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
@@ -314,30 +304,10 @@ public class RangeInputSplit extends InputSplit implements Writable {
     }
   }
 
-  /**
-   * 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;
   }
@@ -350,24 +320,11 @@ public class RangeInputSplit extends InputSplit implements Writable {
     return tableId;
   }
 
-  /**
-   * @see #getInstance(ClientConfiguration)
-   * @deprecated since 1.7.0, use getInstance(ClientConfiguration) instead.
-   */
-  @Deprecated
-  public Instance getInstance() {
-    return getInstance(ClientConfiguration.loadDefault());
-  }
-
   public Instance getInstance(ClientConfiguration base) {
     if (null == instanceName) {
       return null;
     }
 
-    if (isMockInstance()) {
-      return DeprecationUtil.makeMockInstance(getInstanceName());
-    }
-
     if (null == zooKeepers) {
       return null;
     }
@@ -425,22 +382,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.locations = Arrays.copyOf(locations, locations.length);
   }
 
-  /**
-   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
-   */
-  @Deprecated
-  public Boolean isMockInstance() {
-    return mockInstance;
-  }
-
-  /**
-   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
-   */
-  @Deprecated
-  public void setMockInstance(Boolean mockInstance) {
-    this.mockInstance = mockInstance;
-  }
-
   public Boolean isIsolatedScan() {
     return isolatedScan;
   }
@@ -515,7 +456,6 @@ public class RangeInputSplit extends InputSplit implements Writable {
     sb.append(" authenticationTokenFile: ").append(tokenFile);
     sb.append(" Authorizations: ").append(auths);
     sb.append(" offlineScan: ").append(offline);
-    sb.append(" mockInstance: ").append(mockInstance);
     sb.append(" isolatedScan: ").append(isolatedScan);
     sb.append(" localIterators: ").append(localIterators);
     sb.append(" fetchColumns: ").append(fetchedColumns);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
index b81b064..315e40c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
@@ -27,7 +27,6 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 
@@ -41,7 +40,6 @@ public class SplitUtils {
       Authorizations auths, Level logLevel) {
     split.setInstanceName(instance.getInstanceName());
     split.setZooKeepers(instance.getZooKeepers());
-    DeprecationUtil.setMockInstance(split, DeprecationUtil.isMockInstance(instance));
 
     split.setPrincipal(principal);
     split.setToken(token);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
index 942c746..2911c77 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
@@ -37,7 +37,6 @@ import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.impl.DelegationTokenStub;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -320,29 +319,6 @@ public class ConfiguratorBase {
   }
 
   /**
-   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param instanceName
-   *          the Accumulo instance name
-   * @since 1.6.0
-   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
-   */
-  @Deprecated
-  public static void setMockInstance(Class<?> implementingClass, Configuration conf, String instanceName) {
-    String key = enumToConfKey(implementingClass, InstanceOpts.TYPE);
-    if (!conf.get(key, "").isEmpty())
-      throw new IllegalStateException("Instance info can only be set once per job; it has already been configured with " + conf.get(key));
-    conf.set(key, "MockInstance");
-
-    checkArgument(instanceName != null, "instanceName is null");
-    conf.set(enumToConfKey(implementingClass, InstanceOpts.NAME), instanceName);
-  }
-
-  /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    *
    * @param implementingClass
@@ -355,9 +331,7 @@ public class ConfiguratorBase {
    */
   public static Instance getInstance(Class<?> implementingClass, Configuration conf) {
     String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE), "");
-    if ("MockInstance".equals(instanceType))
-      return DeprecationUtil.makeMockInstance(conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME)));
-    else if ("ZooKeeperInstance".equals(instanceType)) {
+    if ("ZooKeeperInstance".equals(instanceType)) {
       return new ZooKeeperInstance(getClientConfiguration(implementingClass, conf));
     } else if (instanceType.isEmpty())
       throw new IllegalStateException("Instance has not been configured for " + implementingClass.getSimpleName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index e80e95b..e07ba32 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -50,12 +50,10 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 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.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
@@ -68,7 +66,6 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.conf.Configuration;
@@ -675,9 +672,6 @@ public class InputConfigurator extends ConfiguratorBase {
    * @since 1.6.0
    */
   public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf, String tableId) throws TableNotFoundException {
-    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
-    if ("MockInstance".equals(instanceType))
-      return DeprecationUtil.makeMockLocator();
     Instance instance = getInstance(implementingClass, conf);
     ClientConfiguration clientConf = getClientConfiguration(implementingClass, conf);
     ClientContext context = new ClientContext(instance,
@@ -744,69 +738,6 @@ public class InputConfigurator extends ConfiguratorBase {
     }
   }
 
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
-   *
-   * <p>
-   * The implementation (JobContext or JobConf which created the Configuration) needs to be used to extract the proper {@link AuthenticationToken} for
-   * {@link DelegationTokenImpl} support.
-   *
-   * @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
-   * @throws IOException
-   *           if the context is improperly configured
-   * @since 1.6.0
-   *
-   * @see #validateInstance(Class, Configuration)
-   * @see #validatePermissions(Class, Configuration, Connector)
-   */
-  @Deprecated
-  public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
-
-    Map<String,InputTableConfig> inputTableConfigs = getInputTableConfigs(implementingClass, conf);
-    if (!isConnectorInfoSet(implementingClass, conf))
-      throw new IOException("Input info has not been set.");
-    String instanceKey = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
-    if (!"MockInstance".equals(instanceKey) && !"ZooKeeperInstance".equals(instanceKey))
-      throw new IOException("Instance info has not been set.");
-    // validate that we can connect as configured
-    try {
-      String principal = getPrincipal(implementingClass, conf);
-      AuthenticationToken token = getAuthenticationToken(implementingClass, conf);
-      Connector c = getInstance(implementingClass, conf).getConnector(principal, token);
-      if (!c.securityOperations().authenticateUser(principal, token))
-        throw new IOException("Unable to authenticate user");
-
-      if (getInputTableConfigs(implementingClass, conf).size() == 0)
-        throw new IOException("No table set.");
-
-      for (Map.Entry<String,InputTableConfig> tableConfig : inputTableConfigs.entrySet()) {
-        if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getKey(), TablePermission.READ))
-          throw new IOException("Unable to access table");
-      }
-      for (Map.Entry<String,InputTableConfig> tableConfigEntry : inputTableConfigs.entrySet()) {
-        InputTableConfig tableConfig = tableConfigEntry.getValue();
-        if (!tableConfig.shouldUseLocalIterators()) {
-          if (tableConfig.getIterators() != null) {
-            for (IteratorSetting iter : tableConfig.getIterators()) {
-              if (!c.tableOperations().testClassLoad(tableConfigEntry.getKey(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
-                throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
-            }
-          }
-        }
-      }
-    } catch (AccumuloException e) {
-      throw new IOException(e);
-    } catch (AccumuloSecurityException e) {
-      throw new IOException(e);
-    } catch (TableNotFoundException e) {
-      throw new IOException(e);
-    }
-  }
-
   /**
    * Returns the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the configuration based on the properties set using the single-table
    * input methods.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
deleted file mode 100644
index 6914071..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.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.client.mapreduce.lib.util;
-
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Level;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link Instance#getConnector(String, AuthenticationToken)}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum ConnectorInfo {
-    IS_CONFIGURED, PRINCIPAL, TOKEN, TOKEN_CLASS
-  }
-
-  /**
-   * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link org.apache.accumulo.core.client.mock.MockInstance}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static enum InstanceOpts {
-    TYPE, NAME, ZOO_KEEPERS;
-  }
-
-  /**
-   * Configuration keys for general configuration options.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static enum GeneralOpts {
-    LOG_LEVEL
-  }
-
-  /**
-   * Provides a configuration key for a given feature enum, prefixed by the implementingClass
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param e
-   *          the enum used to provide the unique part of the configuration key
-   * @return the configuration key
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static String enumToConfKey(Class<?> implementingClass, Enum<?> e) {
-    return implementingClass.getSimpleName() + "." + e.getDeclaringClass().getSimpleName() + "." + StringUtils.camelize(e.name().toLowerCase());
-  }
-
-  /**
-   * Sets the connector information needed to communicate with Accumulo in this job.
-   *
-   * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
-   * conversion to a string, and is not intended to be secure.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param principal
-   *          a valid Accumulo user name
-   * @param token
-   *          the user's password
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setConnectorInfo(Class<?> implementingClass, Configuration conf, String principal, AuthenticationToken token)
-      throws AccumuloSecurityException {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setConnectorInfo(implementingClass, conf, principal, token);
-  }
-
-  /**
-   * Determines if the connector info has already been set for this instance.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the connector info has already been set, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.isConnectorInfoSet(implementingClass, conf);
-  }
-
-  /**
-   * Gets the user name from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the principal
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getPrincipal(implementingClass, conf);
-  }
-
-  /**
-   * DON'T USE THIS. No, really, don't use this. You already have an {@link AuthenticationToken} with
-   * {@link org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase#getAuthenticationToken(Class, Configuration)}. You don't need to construct it
-   * yourself.
-   * <p>
-   * Gets the serialized token class from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the principal
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static String getTokenClass(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getAuthenticationToken(implementingClass, conf).getClass().getName();
-  }
-
-  /**
-   * DON'T USE THIS. No, really, don't use this. You already have an {@link AuthenticationToken} with
-   * {@link org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase#getAuthenticationToken(Class, Configuration)}. You don't need to construct it
-   * yourself.
-   * <p>
-   * Gets the password from the configuration. WARNING: The password is stored in the Configuration and shared with all MapReduce tasks; It is BASE64 encoded to
-   * provide a charset safe conversion to a string, and is not intended to be secure.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the decoded principal's authentication token
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
-   */
-  @Deprecated
-  public static byte[] getToken(Class<?> implementingClass, Configuration conf) {
-    return AuthenticationTokenSerializer.serialize(org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getAuthenticationToken(
-        implementingClass, conf));
-  }
-
-  /**
-   * Configures a {@link ZooKeeperInstance} for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param instanceName
-   *          the Accumulo instance name
-   * @param zooKeepers
-   *          a comma-separated list of zookeeper servers
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf, String instanceName, String zooKeepers) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setZooKeeperInstance(implementingClass, conf,
-        new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param instanceName
-   *          the Accumulo instance name
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setMockInstance(Class<?> implementingClass, Configuration conf, String instanceName) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setMockInstance(implementingClass, conf, instanceName);
-  }
-
-  /**
-   * Initializes an Accumulo {@link Instance} based on the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return an Accumulo instance
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(Class, Configuration, String, String)
-   */
-  @Deprecated
-  public static Instance getInstance(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getInstance(implementingClass, conf);
-  }
-
-  /**
-   * Sets the log level for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param level
-   *          the logging level
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setLogLevel(Class<?> implementingClass, Configuration conf, Level level) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setLogLevel(implementingClass, conf, level);
-  }
-
-  /**
-   * Gets the log level from this configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the log level
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setLogLevel(Class, Configuration, Level)
-   */
-  @Deprecated
-  public static Level getLogLevel(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getLogLevel(implementingClass, conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java
deleted file mode 100644
index b4f6b8a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/FileOutputConfigurator.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     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.client.mapreduce.lib.util;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class FileOutputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link AccumuloConfiguration}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum Opts {
-    ACCUMULO_PROPERTIES;
-  }
-
-  /**
-   * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the RecordWriter.<br>
-   * These properties correspond to the supported public static setter methods available to this class.
-   *
-   * @param property
-   *          the Accumulo property to check
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  protected static Boolean isSupportedAccumuloProperty(Property property) {
-    switch (property) {
-      case TABLE_FILE_COMPRESSION_TYPE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
-      case TABLE_FILE_BLOCK_SIZE:
-      case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
-      case TABLE_FILE_REPLICATION:
-        return true;
-      default:
-        return false;
-    }
-  }
-
-  /**
-   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
-   * stored in the Job's configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.getAccumuloConfiguration(implementingClass, conf);
-  }
-
-  /**
-   * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param compressionType
-   *          one of "none", "gz", "lzo", or "snappy"
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setCompressionType(Class<?> implementingClass, Configuration conf, String compressionType) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setCompressionType(implementingClass, conf, compressionType);
-  }
-
-  /**
-   * Sets the size for data blocks within each file.<br>
-   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as a group.
-   *
-   * <p>
-   * Making this value smaller may increase seek performance, but at the cost of increasing the size of the indexes (which can also affect seek performance).
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param dataBlockSize
-   *          the block size, in bytes
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setDataBlockSize(Class<?> implementingClass, Configuration conf, long dataBlockSize) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setDataBlockSize(implementingClass, conf, dataBlockSize);
-  }
-
-  /**
-   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by the underlying file system.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param fileBlockSize
-   *          the block size, in bytes
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setFileBlockSize(Class<?> implementingClass, Configuration conf, long fileBlockSize) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setFileBlockSize(implementingClass, conf, fileBlockSize);
-  }
-
-  /**
-   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy within the file, while larger blocks mean a more shallow
-   * index hierarchy within the file. This can affect the performance of queries.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param indexBlockSize
-   *          the block size, in bytes
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf, long indexBlockSize) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setIndexBlockSize(implementingClass, conf, indexBlockSize);
-  }
-
-  /**
-   * Sets the file system replication factor for the resulting file, overriding the file system default.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param replication
-   *          the number of replicas for produced files
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setReplication(Class<?> implementingClass, Configuration conf, int replication) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setReplication(implementingClass, conf, replication);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
deleted file mode 100644
index b85253c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ /dev/null
@@ -1,461 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     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.client.mapreduce.lib.util;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-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.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-
-/**
- * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
- * @since 1.5.0
- */
-@Deprecated
-public class InputConfigurator extends ConfiguratorBase {
-
-  /**
-   * Configuration keys for {@link Scanner}.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum ScanOpts {
-    TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS
-  }
-
-  /**
-   * Configuration keys for various features.
-   *
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static enum Features {
-    AUTO_ADJUST_RANGES, SCAN_ISOLATION, USE_LOCAL_ITERATORS, SCAN_OFFLINE
-  }
-
-  /**
-   * Sets the name of the input table, over which this job will scan.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param tableName
-   *          the table to use when the tablename is null in the write call
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setInputTableName(implementingClass, conf, tableName);
-  }
-
-  /**
-   * Gets the table name from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the table name
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setInputTableName(Class, Configuration, String)
-   */
-  @Deprecated
-  public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getInputTableName(implementingClass, conf);
-  }
-
-  /**
-   * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param auths
-   *          the user's authorizations
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf, Authorizations auths) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setScanAuthorizations(implementingClass, conf, auths);
-  }
-
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the Accumulo scan authorizations
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setScanAuthorizations(Class, Configuration, Authorizations)
-   */
-  @Deprecated
-  public static Authorizations getScanAuthorizations(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getScanAuthorizations(implementingClass, conf);
-  }
-
-  /**
-   * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param ranges
-   *          the ranges that will be mapped over
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setRanges(implementingClass, conf, ranges);
-  }
-
-  /**
-   * Gets the ranges to scan over from a job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return the ranges
-   * @throws IOException
-   *           if the ranges have been encoded improperly
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setRanges(Class, Configuration, Collection)
-   */
-  @Deprecated
-  public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getRanges(implementingClass, conf);
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param columnFamilyColumnQualifierPairs
-   *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
-   *          selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.fetchColumns(implementingClass, conf, columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   */
-  @Deprecated
-  public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.serializeColumns(columnFamilyColumnQualifierPairs);
-  }
-
-  /**
-   * Gets the columns to be mapped over from this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return a set of columns
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #fetchColumns(Class, Configuration, Collection)
-   */
-  @Deprecated
-  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getFetchedColumns(implementingClass, conf);
-  }
-
-  /**
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   */
-  @Deprecated
-  public static Set<Pair<Text,Text>> deserializeFetchedColumns(Collection<String> serialized) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.deserializeFetchedColumns(serialized);
-  }
-
-  /**
-   * Encode an iterator on the input for this job.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param cfg
-   *          the configuration of the iterator
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void addIterator(Class<?> implementingClass, Configuration conf, IteratorSetting cfg) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.addIterator(implementingClass, conf, cfg);
-  }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return a list of iterators
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #addIterator(Class, Configuration, IteratorSetting)
-   */
-  @Deprecated
-  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getIterators(implementingClass, conf);
-  }
-
-  /**
-   * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
-   * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
-   *
-   * <p>
-   * By default, this feature is <b>enabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @see #setRanges(Class, Configuration, Collection)
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setAutoAdjustRanges(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has auto-adjust ranges enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return false if the feature is disabled, true otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setAutoAdjustRanges(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getAutoAdjustRanges(implementingClass, conf);
-  }
-
-  /**
-   * Controls the use of the {@link IsolatedScanner} in this job.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setScanIsolation(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has isolation enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setScanIsolation(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.isIsolated(implementingClass, conf);
-  }
-
-  /**
-   * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
-   * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setLocalIterators(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration uses local iterators.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setLocalIterators(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.usesLocalIterators(implementingClass, conf);
-  }
-
-  /**
-   * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
-   * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
-   * fail.
-   *
-   * <p>
-   * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
-   *
-   * <p>
-   * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
-   * on the mapper's classpath. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
-   *
-   * <p>
-   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
-   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
-   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
-   *
-   * <p>
-   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
-   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
-   *
-   * <p>
-   * By default, this feature is <b>disabled</b>.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param enableFeature
-   *          the feature is enabled if true, disabled otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setOfflineTableScan(implementingClass, conf, enableFeature);
-  }
-
-  /**
-   * Determines whether a configuration has the offline table scan feature enabled.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return true if the feature is enabled, false otherwise
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   * @see #setOfflineTableScan(Class, Configuration, boolean)
-   */
-  @Deprecated
-  public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.isOfflineScan(implementingClass, conf);
-  }
-
-  /**
-   * 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
-   * @return an Accumulo tablet locator
-   * @throws TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf) throws TableNotFoundException {
-    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getTabletLocator(implementingClass, conf,
-        Tables.getTableId(getInstance(implementingClass, conf), getInputTableName(implementingClass, conf)));
-  }
-
-  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
-  /**
-   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
-   *
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @throws IOException
-   *           if the context is improperly configured
-   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
-   * @since 1.5.0
-   */
-  @Deprecated
-  public static void validateOptions(Class<?> implementingClass, Configuration conf) throws IOException {
-    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.validateOptions(implementingClass, conf);
-  }
-
-}