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 2014/04/11 21:49:20 UTC

[2/4] ACCUMULO-2659 Properly deprecate public API changes to mapred utils

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
new file mode 100644
index 0000000..0ba42cd
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.impl;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * @since 1.6.0
+ */
+public class OutputConfigurator extends ConfiguratorBase {
+
+  /**
+   * Configuration keys for {@link BatchWriter}.
+   * 
+   * @since 1.6.0
+   */
+  public static enum WriteOpts {
+    DEFAULT_TABLE_NAME, BATCH_WRITER_CONFIG
+  }
+
+  /**
+   * Configuration keys for various features.
+   * 
+   * @since 1.6.0
+   */
+  public static enum Features {
+    CAN_CREATE_TABLES, SIMULATION_MODE
+  }
+
+  /**
+   * Sets the default table name to use if one emits a null in place of a table name for a given mutation. Table names can only be alpha-numeric and
+   * underscores.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param tableName
+   *          the table to use when the tablename is null in the write call
+   * @since 1.6.0
+   */
+  public static void setDefaultTableName(Class<?> implementingClass, Configuration conf, String tableName) {
+    if (tableName != null)
+      conf.set(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME), tableName);
+  }
+
+  /**
+   * Gets the default table name from the configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the default table name
+   * @since 1.6.0
+   * @see #setDefaultTableName(Class, Configuration, String)
+   */
+  public static String getDefaultTableName(Class<?> implementingClass, Configuration conf) {
+    return conf.get(enumToConfKey(implementingClass, WriteOpts.DEFAULT_TABLE_NAME));
+  }
+
+  /**
+   * Sets the configuration for for the job's {@link BatchWriter} instances. If not set, a new {@link BatchWriterConfig}, with sensible built-in defaults is
+   * used. Setting the configuration multiple times overwrites any previous configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param bwConfig
+   *          the configuration for the {@link BatchWriter}
+   * @since 1.6.0
+   */
+  public static void setBatchWriterOptions(Class<?> implementingClass, Configuration conf, BatchWriterConfig bwConfig) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    String serialized;
+    try {
+      bwConfig.write(new DataOutputStream(baos));
+      serialized = new String(baos.toByteArray(), Constants.UTF8);
+      baos.close();
+    } catch (IOException e) {
+      throw new IllegalArgumentException("unable to serialize " + BatchWriterConfig.class.getName());
+    }
+    conf.set(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG), serialized);
+  }
+
+  /**
+   * Gets the {@link BatchWriterConfig} settings.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return the configuration object
+   * @since 1.6.0
+   * @see #setBatchWriterOptions(Class, Configuration, BatchWriterConfig)
+   */
+  public static BatchWriterConfig getBatchWriterOptions(Class<?> implementingClass, Configuration conf) {
+    String serialized = conf.get(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG));
+    BatchWriterConfig bwConfig = new BatchWriterConfig();
+    if (serialized == null || serialized.isEmpty()) {
+      return bwConfig;
+    } else {
+      try {
+        ByteArrayInputStream bais = new ByteArrayInputStream(serialized.getBytes(Constants.UTF8));
+        bwConfig.readFields(new DataInputStream(bais));
+        bais.close();
+        return bwConfig;
+      } catch (IOException e) {
+        throw new IllegalArgumentException("unable to serialize " + BatchWriterConfig.class.getName());
+      }
+    }
+  }
+
+  /**
+   * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric and underscores.
+   * 
+   * <p>
+   * By default, this feature is <b>disabled</b>.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setCreateTables(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), enableFeature);
+  }
+
+  /**
+   * Determines whether tables are permitted to be created as needed.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is disabled, false otherwise
+   * @since 1.6.0
+   * @see #setCreateTables(Class, Configuration, boolean)
+   */
+  public static Boolean canCreateTables(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), false);
+  }
+
+  /**
+   * Sets the directive to use simulation mode for this job. In simulation mode, no output is produced. This is useful for testing.
+   * 
+   * <p>
+   * By default, this feature is <b>disabled</b>.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param enableFeature
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public static void setSimulationMode(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+    conf.setBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), enableFeature);
+  }
+
+  /**
+   * Determines whether this feature is enabled.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setSimulationMode(Class, Configuration, boolean)
+   */
+  public static Boolean getSimulationMode(Class<?> implementingClass, Configuration conf) {
+    return conf.getBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java
new file mode 100644
index 0000000..243160d
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/package-info.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.client.mapreduce.lib.impl;
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
----------------------------------------------------------------------
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 1569579..54730ef 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
@@ -27,7 +27,7 @@ import java.util.Scanner;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.mapreduce.lib.util.DistributedCacheHelper;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.DistributedCacheHelper;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/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
index 54ff976..68cd389 100644
--- 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
@@ -16,11 +16,6 @@
  */
 package org.apache.accumulo.core.client.mapreduce.lib.util;
 
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Instance;
@@ -28,60 +23,47 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 
 /**
+ * @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,
-  }
-
-  public static enum TokenSource {
-    FILE, INLINE;
-
-    private String prefix;
-
-    private TokenSource() {
-      prefix = name().toLowerCase() + ":";
-    }
-
-    public String prefix() {
-      return prefix;
-    }
+    IS_CONFIGURED, PRINCIPAL, TOKEN, TOKEN_CLASS
   }
 
   /**
    * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link MockInstance}.
    * 
+   * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
    * @since 1.5.0
    */
-  public static enum InstanceOpts {
-    TYPE, NAME, ZOO_KEEPERS, CLIENT_CONFIG;
+  @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
    */
-  public static enum GeneralOpts {
+  @Deprecated
+  protected static enum GeneralOpts {
     LOG_LEVEL
   }
 
@@ -93,8 +75,10 @@ public class ConfiguratorBase {
    * @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());
   }
@@ -114,52 +98,13 @@ public class ConfiguratorBase {
    *          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 {
-    if (isConnectorInfoSet(implementingClass, conf))
-      throw new IllegalStateException("Connector info for " + implementingClass.getSimpleName() + " can only be set once per job");
-
-    ArgumentChecker.notNull(principal, token);
-    conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
-    conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN),
-        TokenSource.INLINE.prefix() + token.getClass().getName() + ":" + Base64.encodeBase64String(AuthenticationTokenSerializer.serialize(token)));
-  }
-
-  /**
-   * 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");
-
-    ArgumentChecker.notNull(principal, tokenFile);
-
-    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);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setConnectorInfo(implementingClass, conf, principal, token);
   }
 
   /**
@@ -170,11 +115,13 @@ public class ConfiguratorBase {
    * @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 conf.getBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), false);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.isConnectorInfoSet(implementingClass, conf);
   }
 
   /**
@@ -185,83 +132,57 @@ public class ConfiguratorBase {
    * @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 conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getPrincipal(implementingClass, conf);
   }
 
   /**
-   * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
+   * 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's authentication token
-   * @since 1.6.0
+   * @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)
-   * @see #setConnectorInfo(Class, Configuration, String, String)
    */
-  public static AuthenticationToken getAuthenticationToken(Class<?> implementingClass, Configuration conf) {
-    String token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
-    if (token == null || token.isEmpty())
-      return null;
-    if (token.startsWith(TokenSource.INLINE.prefix())) {
-      String[] args = token.substring(TokenSource.INLINE.prefix().length()).split(":", 2);
-      if (args.length == 2)
-        return AuthenticationTokenSerializer.deserialize(args[0], Base64.decodeBase64(args[1].getBytes(Constants.UTF8)));
-    } else if (token.startsWith(TokenSource.FILE.prefix())) {
-      String tokenFileName = token.substring(TokenSource.FILE.prefix().length());
-      return getTokenFromFile(conf, getPrincipal(implementingClass, conf), tokenFileName);
-    }
-
-    throw new IllegalStateException("Token was not properly serialized into the configuration");
+  @Deprecated
+  public static String getTokenClass(Class<?> implementingClass, Configuration conf) {
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getAuthenticationToken(implementingClass, conf).getClass().getName();
   }
 
   /**
-   * Reads from the token file in distributed cache. Currently, the token file stores data separated by colons e.g. principal:token_class:token
+   * 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 context for the configured job
-   * @return path to the token file as a String
-   * @since 1.6.0
+   *          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)
    */
-  public static AuthenticationToken getTokenFromFile(Configuration conf, String principal, String tokenFile) {
-    FSDataInputStream in = null;
-    try {
-      URI[] uris = DistributedCacheHelper.getCacheFiles(conf);
-      Path path = null;
-      for (URI u : uris) {
-        if (u.toString().equals(tokenFile)) {
-          path = new Path(u);
-        }
-      }
-      if (path == null) {
-        throw new IllegalArgumentException("Couldn't find password file called \"" + tokenFile + "\" in cache.");
-      }
-      FileSystem fs = FileSystem.get(conf);
-      in = fs.open(path);
-    } catch (IOException e) {
-      throw new IllegalArgumentException("Couldn't open password file called \"" + tokenFile + "\".");
-    }
-    java.util.Scanner fileScanner = new java.util.Scanner(in);
-    try {
-      while (fileScanner.hasNextLine()) {
-        Credentials creds = Credentials.deserialize(fileScanner.nextLine());
-        if (principal.equals(creds.getPrincipal())) {
-          return creds.getToken();
-        }
-      }
-      throw new IllegalArgumentException("Couldn't find token for user \"" + principal + "\" in file \"" + tokenFile + "\"");
-    } finally {
-      if (fileScanner != null && fileScanner.ioException() == null)
-        fileScanner.close();
-      else if (fileScanner.ioException() != null)
-        throw new RuntimeException(fileScanner.ioException());
-    }
+  @Deprecated
+  public static byte[] getToken(Class<?> implementingClass, Configuration conf) {
+    return AuthenticationTokenSerializer.serialize(org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getAuthenticationToken(
+        implementingClass, conf));
   }
 
   /**
@@ -275,35 +196,13 @@ public class ConfiguratorBase {
    *          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 since 1.6.0; Use {@link #setZooKeeperInstance(Class, Configuration, ClientConfiguration)} instead.
    */
-
   @Deprecated
   public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf, String instanceName, String zooKeepers) {
-    ArgumentChecker.notNull(instanceName, zooKeepers);
-    setZooKeeperInstance(implementingClass, conf, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
-  }
-
-  /**
-   * 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 clientConfig
-   *          client configuration for specifying connection timeouts, SSL connection options, etc.
-   * @since 1.5.0
-   */
-  public static void setZooKeeperInstance(Class<?> implementingClass, Configuration conf, ClientConfiguration clientConfig) {
-    String key = enumToConfKey(implementingClass, InstanceOpts.TYPE);
-    if (!conf.get(key, "").isEmpty())
-      throw new IllegalStateException("Instance info can only be set once per job; it has already been configured with " + conf.get(key));
-    conf.set(key, "ZooKeeperInstance");
-    if (clientConfig != null) {
-      conf.set(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG), clientConfig.serialize());
-    }
+    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setZooKeeperInstance(implementingClass, conf,
+        new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
@@ -315,16 +214,12 @@ public class ConfiguratorBase {
    *          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) {
-    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");
-
-    ArgumentChecker.notNull(instanceName);
-    conf.set(enumToConfKey(implementingClass, InstanceOpts.NAME), instanceName);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setMockInstance(implementingClass, conf, instanceName);
   }
 
   /**
@@ -335,27 +230,14 @@ public class ConfiguratorBase {
    * @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, ClientConfiguration)
+   * @see #setZooKeeperInstance(Class, Configuration, String, String)
    * @see #setMockInstance(Class, Configuration, String)
    */
+  @Deprecated
   public static Instance getInstance(Class<?> implementingClass, Configuration conf) {
-    String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE), "");
-    if ("MockInstance".equals(instanceType))
-      return new MockInstance(conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME)));
-    else if ("ZooKeeperInstance".equals(instanceType)) {
-      String clientConfigString = conf.get(enumToConfKey(implementingClass, InstanceOpts.CLIENT_CONFIG));
-      if (clientConfigString == null) {
-        String instanceName = conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME));
-        String zookeepers = conf.get(enumToConfKey(implementingClass, InstanceOpts.ZOO_KEEPERS));
-        return new ZooKeeperInstance(ClientConfiguration.loadDefault().withInstance(instanceName).withZkHosts(zookeepers));
-      } else {
-        return new ZooKeeperInstance(ClientConfiguration.deserialize(clientConfigString));
-      }
-    } else if (instanceType.isEmpty())
-      throw new IllegalStateException("Instance has not been configured for " + implementingClass.getSimpleName());
-    else
-      throw new IllegalStateException("Unrecognized instance type " + instanceType);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getInstance(implementingClass, conf);
   }
 
   /**
@@ -367,12 +249,12 @@ public class ConfiguratorBase {
    *          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) {
-    ArgumentChecker.notNull(level);
-    Logger.getLogger(implementingClass).setLevel(level);
-    conf.setInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), level.toInt());
+    org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.setLogLevel(implementingClass, conf, level);
   }
 
   /**
@@ -383,11 +265,13 @@ public class ConfiguratorBase {
    * @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 Level.toLevel(conf.getInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), Level.INFO.toInt()));
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.getLogLevel(implementingClass, conf);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/DistributedCacheHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/DistributedCacheHelper.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/DistributedCacheHelper.java
deleted file mode 100644
index 20f8719..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/DistributedCacheHelper.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     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.net.URI;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.filecache.DistributedCache;
-import org.apache.hadoop.fs.Path;
-
-@SuppressWarnings("deprecation")
-public class DistributedCacheHelper {
-  
-  public static void addCacheFile(URI uri, Configuration conf) {
-    DistributedCache.addCacheFile(uri, conf);
-  }
-  
-  public static URI[] getCacheFiles(Configuration conf) throws IOException {
-    return DistributedCache.getCacheFiles(conf);
-  }
-  
-  public static Path[] getLocalCacheFiles(Configuration conf) throws IOException {
-    return DistributedCache.getLocalCacheFiles(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/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
index 7c49f79..5b431c3 100644
--- 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
@@ -16,36 +16,38 @@
  */
 package org.apache.accumulo.core.client.mapreduce.lib.util;
 
-import java.util.Arrays;
-import java.util.Map.Entry;
-
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
 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:
@@ -58,31 +60,7 @@ public class FileOutputConfigurator extends ConfiguratorBase {
         return false;
     }
   }
-  
-  /**
-   * Helper for transforming Accumulo configuration properties into something that can be stored safely inside the Hadoop Job configuration.
-   * 
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param property
-   *          the supported Accumulo property
-   * @param value
-   *          the value of the property to set
-   * @since 1.5.0
-   */
-  private static <T> void setAccumuloProperty(Class<?> implementingClass, Configuration conf, Property property, T value) {
-    if (isSupportedAccumuloProperty(property)) {
-      String val = String.valueOf(value);
-      if (property.getType().isValidFormat(val))
-        conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + property.getKey(), val);
-      else
-        throw new IllegalArgumentException("Value is not appropriate for property type '" + property.getType() + "'");
-    } else
-      throw new IllegalArgumentException("Unsupported configuration property " + property.getKey());
-  }
-  
+
   /**
    * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
    * stored in the Job's configuration.
@@ -91,17 +69,14 @@ public class FileOutputConfigurator extends ConfiguratorBase {
    *          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) {
-    String prefix = enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + ".";
-    ConfigurationCopy acuConf = new ConfigurationCopy(AccumuloConfiguration.getDefaultConfiguration());
-    for (Entry<String,String> entry : conf)
-      if (entry.getKey().startsWith(prefix))
-        acuConf.set(Property.getPropertyByKey(entry.getKey().substring(prefix.length())), entry.getValue());
-    return acuConf;
+    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.
    * 
@@ -111,14 +86,14 @@ public class FileOutputConfigurator extends ConfiguratorBase {
    *          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) {
-    if (compressionType == null || !Arrays.asList("none", "gz", "lzo", "snappy").contains(compressionType))
-      throw new IllegalArgumentException("Compression type must be one of: none, gz, lzo, snappy");
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSION_TYPE, 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.
@@ -132,12 +107,14 @@ public class FileOutputConfigurator extends ConfiguratorBase {
    *          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) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, 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.
    * 
@@ -147,12 +124,14 @@ public class FileOutputConfigurator extends ConfiguratorBase {
    *          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) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_BLOCK_SIZE, 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.
@@ -163,12 +142,14 @@ public class FileOutputConfigurator extends ConfiguratorBase {
    *          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) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, 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.
    * 
@@ -178,10 +159,12 @@ public class FileOutputConfigurator extends ConfiguratorBase {
    *          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) {
-    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_REPLICATION, replication);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator.setReplication(implementingClass, conf, replication);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/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
index c76a185..3b1feb6 100644
--- 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
@@ -16,80 +16,49 @@
  */
 package org.apache.accumulo.core.client.mapreduce.lib.util;
 
-import static org.apache.accumulo.core.util.ArgumentChecker.notNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
-import java.util.StringTokenizer;
 
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mock.MockTabletLocator;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.TextUtil;
-import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.collect.Maps;
 
 /**
+ * @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, TABLE_CONFIGS
+    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
   }
@@ -103,24 +72,29 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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) {
-    notNull(tableName);
-    conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setInputTableName(implementingClass, conf, tableName);
   }
 
   /**
-   * Sets the name of the input table, over which this job will scan.
+   * 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 conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getInputTableName(implementingClass, conf);
   }
 
   /**
@@ -132,11 +106,12 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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) {
-    if (auths != null && !auths.isEmpty())
-      conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setScanAuthorizations(implementingClass, conf, auths);
   }
 
   /**
@@ -147,16 +122,17 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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) {
-    String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
-    return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes(Constants.UTF8));
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getScanAuthorizations(implementingClass, conf);
   }
 
   /**
-   * Sets the input ranges to scan on all input tables for this job. If not set, the entire table will be scanned.
+   * 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
@@ -164,24 +140,12 @@ public class InputConfigurator extends ConfiguratorBase {
    *          the Hadoop configuration object to configure
    * @param ranges
    *          the ranges that will be mapped over
-   * @throws IllegalArgumentException
-   *           if the ranges cannot be encoded into base 64
+   * @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) {
-    notNull(ranges);
-
-    ArrayList<String> rangeStrings = new ArrayList<String>(ranges.size());
-    try {
-      for (Range r : ranges) {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        r.write(new DataOutputStream(baos));
-        rangeStrings.add(new String(Base64.encodeBase64(baos.toByteArray())));
-      }
-      conf.setStrings(enumToConfKey(implementingClass, ScanOpts.RANGES), rangeStrings.toArray(new String[0]));
-    } catch (IOException ex) {
-      throw new IllegalArgumentException("Unable to encode ranges to Base64", ex);
-    }
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setRanges(implementingClass, conf, ranges);
   }
 
   /**
@@ -194,58 +158,17 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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 {
-
-    Collection<String> encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
-    List<Range> ranges = new ArrayList<Range>();
-    for (String rangeString : encodedRanges) {
-      ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(rangeString.getBytes()));
-      Range range = new Range();
-      range.readFields(new DataInputStream(bais));
-      ranges.add(range);
-    }
-    return ranges;
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getRanges(implementingClass, conf);
   }
 
   /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
-   * 
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return a list of iterators
-   * @since 1.5.0
-   * @see #addIterator(Class, Configuration, IteratorSetting)
-   */
-  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
-    String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
-
-    // If no iterators are present, return an empty list
-    if (iterators == null || iterators.isEmpty())
-      return new ArrayList<IteratorSetting>();
-
-    // Compose the set of iterators encoded in the job configuration
-    StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
-    List<IteratorSetting> list = new ArrayList<IteratorSetting>();
-    try {
-      while (tokens.hasMoreTokens()) {
-        String itstring = tokens.nextToken();
-        ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(itstring.getBytes()));
-        list.add(new IteratorSetting(new DataInputStream(bais)));
-        bais.close();
-      }
-    } catch (IOException e) {
-      throw new IllegalArgumentException("couldn't decode iterator settings");
-    }
-    return list;
-  }
-
-  /**
-   * Restricts the columns that will be mapped over for the single input table on this job.
+   * 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
@@ -254,31 +177,20 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param columnFamilyColumnQualifierPairs
    *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
    *          selected. An empty set is the default and is equivalent to scanning the all columns.
-   * @throws IllegalArgumentException
-   *           if the column family is null
+   * @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) {
-    notNull(columnFamilyColumnQualifierPairs);
-    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
-    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
+    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) {
-    notNull(columnFamilyColumnQualifierPairs);
-    ArrayList<String> columnStrings = new ArrayList<String>(columnFamilyColumnQualifierPairs.size());
-    for (Pair<Text,Text> column : columnFamilyColumnQualifierPairs) {
-
-      if (column.getFirst() == null)
-        throw new IllegalArgumentException("Column family can not be null");
-
-      String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), Constants.UTF8);
-      if (column.getSecond() != null)
-        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), Constants.UTF8);
-      columnStrings.add(col);
-    }
-
-    return columnStrings.toArray(new String[0]);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.serializeColumns(columnFamilyColumnQualifierPairs);
   }
 
   /**
@@ -289,40 +201,25 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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) {
-    notNull(conf);
-    String confValue = conf.get(enumToConfKey(implementingClass, ScanOpts.COLUMNS));
-    List<String> serialized = new ArrayList<String>();
-    if (confValue != null) {
-      // Split and include any trailing empty strings to allow empty column families
-      for (String val : confValue.split(",", -1)) {
-        serialized.add(val);
-      }
-    }
-    return deserializeFetchedColumns(serialized);
+    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) {
-    Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
-
-    if (null == serialized) {
-      return columns;
-    }
-
-    for (String col : serialized) {
-      int idx = col.indexOf(":");
-      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Constants.UTF8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(Constants.UTF8)));
-      Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes(Constants.UTF8)));
-      columns.add(new Pair<Text,Text>(cf, cq));
-    }
-    return columns;
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.deserializeFetchedColumns(serialized);
   }
 
   /**
-   * Encode an iterator on the input for the single input table associated with this job.
+   * 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
@@ -330,32 +227,29 @@ public class InputConfigurator extends ConfiguratorBase {
    *          the Hadoop configuration object to configure
    * @param cfg
    *          the configuration of the iterator
-   * @throws IllegalArgumentException
-   *           if the iterator can't be serialized into the configuration
+   * @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) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    String newIter;
-    try {
-      cfg.write(new DataOutputStream(baos));
-      newIter = new String(Base64.encodeBase64(baos.toByteArray()), Constants.UTF8);
-      baos.close();
-    } catch (IOException e) {
-      throw new IllegalArgumentException("unable to serialize IteratorSetting");
-    }
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.addIterator(implementingClass, conf, cfg);
+  }
 
-    String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
-    String iterators = conf.get(confKey);
-    // No iterators specified yet, create a new string
-    if (iterators == null || iterators.isEmpty()) {
-      iterators = newIter;
-    } else {
-      // append the next iterator & reset
-      iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
-    }
-    // Store the iterators w/ the job
-    conf.set(confKey, iterators);
+  /**
+   * 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);
   }
 
   /**
@@ -372,10 +266,12 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setAutoAdjustRanges(implementingClass, conf, enableFeature);
   }
 
   /**
@@ -386,11 +282,13 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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 conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.getAutoAdjustRanges(implementingClass, conf);
   }
 
   /**
@@ -405,10 +303,12 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setScanIsolation(implementingClass, conf, enableFeature);
   }
 
   /**
@@ -419,11 +319,13 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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 conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.isIsolated(implementingClass, conf);
   }
 
   /**
@@ -439,10 +341,12 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setLocalIterators(implementingClass, conf, enableFeature);
   }
 
   /**
@@ -453,11 +357,13 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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 conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.usesLocalIterators(implementingClass, conf);
   }
 
   /**
@@ -471,7 +377,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * 
    * <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.
+   * 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
@@ -491,10 +397,12 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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) {
-    conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.setOfflineTableScan(implementingClass, conf, enableFeature);
   }
 
   /**
@@ -505,88 +413,13 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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 conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
-  }
-
-  /**
-   * Sets configurations for multiple tables at a time.
-   * 
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param configs
-   *          an array of {@link InputTableConfig} objects to associate with the job
-   * @since 1.6.0
-   */
-  public static void setInputTableConfigs(Class<?> implementingClass, Configuration conf, Map<String,InputTableConfig> configs) {
-    MapWritable mapWritable = new MapWritable();
-    for (Map.Entry<String,InputTableConfig> tableConfig : configs.entrySet())
-      mapWritable.put(new Text(tableConfig.getKey()), tableConfig.getValue());
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    try {
-      mapWritable.write(new DataOutputStream(baos));
-    } catch (IOException e) {
-      throw new IllegalStateException("Table configuration could not be serialized.");
-    }
-
-    String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
-    conf.set(confKey, new String(Base64.encodeBase64(baos.toByteArray())));
-  }
-
-  /**
-   * Returns all {@link InputTableConfig} objects associated with this job.
-   * 
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @return all of the table query configs for the job
-   * @since 1.6.0
-   */
-  public static Map<String,InputTableConfig> getInputTableConfigs(Class<?> implementingClass, Configuration conf) {
-    Map<String,InputTableConfig> configs = new HashMap<String,InputTableConfig>();
-    Map.Entry<String,InputTableConfig> defaultConfig = getDefaultInputTableConfig(implementingClass, conf);
-    if (defaultConfig != null)
-      configs.put(defaultConfig.getKey(), defaultConfig.getValue());
-    String configString = conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
-    MapWritable mapWritable = new MapWritable();
-    if (configString != null) {
-      try {
-        byte[] bytes = Base64.decodeBase64(configString.getBytes());
-        ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-        mapWritable.readFields(new DataInputStream(bais));
-        bais.close();
-      } catch (IOException e) {
-        throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
-      }
-    }
-    for (Map.Entry<Writable,Writable> entry : mapWritable.entrySet())
-      configs.put(((Text) entry.getKey()).toString(), (InputTableConfig) entry.getValue());
-
-    return configs;
-  }
-
-  /**
-   * Returns the {@link InputTableConfig} for the given table
-   * 
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop configuration object to configure
-   * @param tableName
-   *          the table name for which to fetch the table query config
-   * @return the table query config for the given table name (if it exists) and null if it does not
-   * @since 1.6.0
-   */
-  public static InputTableConfig getInputTableConfig(Class<?> implementingClass, Configuration conf, String tableName) {
-    Map<String,InputTableConfig> queryConfigs = getInputTableConfigs(implementingClass, conf);
-    return queryConfigs.get(tableName);
+    return org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.isOfflineScan(implementingClass, conf);
   }
 
   /**
@@ -596,19 +429,16 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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
    * @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
    */
-  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 new MockTabletLocator();
-    Instance instance = getInstance(implementingClass, conf);
-    return TabletLocator.getLocator(instance, new Text(tableId));
+  @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)
@@ -621,175 +451,12 @@ public class InputConfigurator extends ConfiguratorBase {
    *          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 {
-
-    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);
-    }
+    org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator.validateOptions(implementingClass, conf);
   }
 
-  /**
-   * Returns the {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} for the configuration based on the properties set using the single-table
-   * input methods.
-   * 
-   * @param implementingClass
-   *          the class whose name will be used as a prefix for the property configuration key
-   * @param conf
-   *          the Hadoop instance for which to retrieve the configuration
-   * @return the config object built from the single input table properties set on the job
-   * @since 1.6.0
-   */
-  protected static Map.Entry<String,InputTableConfig> getDefaultInputTableConfig(Class<?> implementingClass, Configuration conf) {
-    String tableName = getInputTableName(implementingClass, conf);
-    if (tableName != null) {
-      InputTableConfig queryConfig = new InputTableConfig();
-      List<IteratorSetting> itrs = getIterators(implementingClass, conf);
-      if (itrs != null)
-        queryConfig.setIterators(itrs);
-      Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
-      if (columns != null)
-        queryConfig.fetchColumns(columns);
-      List<Range> ranges = null;
-      try {
-        ranges = getRanges(implementingClass, conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      if (ranges != null)
-        queryConfig.setRanges(ranges);
-
-      queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf)).setUseIsolatedScanners(isIsolated(implementingClass, conf))
-          .setUseLocalIterators(usesLocalIterators(implementingClass, conf)).setOfflineScan(isOfflineScan(implementingClass, conf));
-      return Maps.immutableEntry(tableName, queryConfig);
-    }
-    return null;
-  }
-
-  public static Map<String,Map<KeyExtent,List<Range>>> binOffline(String tableId, List<Range> ranges, Instance instance, Connector conn)
-      throws AccumuloException, TableNotFoundException {
-    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-
-    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
-      Tables.clearCache(instance);
-      if (Tables.getTableState(instance, 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(new Text(tableId), startRow, null).getMetadataEntry(), true, null, false);
-      Scanner scanner = conn.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 (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
-            extent = new KeyExtent(key.getRow(), entry.getValue());
-          }
-
-        }
-
-        if (location != null)
-          return null;
-
-        if (!extent.getTableId().toString().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);
-        }
-
-        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
-        if (tabletRanges == null) {
-          tabletRanges = new HashMap<KeyExtent,List<Range>>();
-          binnedRanges.put(last, tabletRanges);
-        }
-
-        List<Range> rangeList = tabletRanges.get(extent);
-        if (rangeList == null) {
-          rangeList = new ArrayList<Range>();
-          tabletRanges.put(extent, rangeList);
-        }
-
-        rangeList.add(range);
-
-        if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
-          break;
-        }
-
-        lastExtent = extent;
-      }
-
-    }
-    return binnedRanges;
-  }
 }