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 22:33:16 UTC

[7/7] git commit: Merge branch '1.6.0-SNAPSHOT'

Merge branch '1.6.0-SNAPSHOT'

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
	core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
	core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
	core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/OutputConfigurator.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/bf102d07
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/bf102d07
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/bf102d07

Branch: refs/heads/master
Commit: bf102d0711103e903afa0589500f5796ad51c366
Parents: b9e49cb f51d7bb
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Apr 11 16:33:00 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Apr 11 16:33:00 2014 -0400

----------------------------------------------------------------------
 CHANGES                                         |   6 +-
 .../core/client/mapred/AbstractInputFormat.java |  64 +-
 .../client/mapred/AccumuloFileOutputFormat.java |   2 +-
 .../mapred/AccumuloMultiTableInputFormat.java   |   2 +-
 .../client/mapred/AccumuloOutputFormat.java     |  22 +-
 .../core/client/mapred/InputFormatBase.java     |   2 +-
 .../client/mapreduce/AbstractInputFormat.java   |  45 +-
 .../mapreduce/AccumuloFileOutputFormat.java     |   2 +-
 .../AccumuloMultiTableInputFormat.java          |   2 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |  19 +-
 .../core/client/mapreduce/InputFormatBase.java  |   2 +-
 .../core/client/mapreduce/RangeInputSplit.java  |   4 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java    | 375 +++++++++
 .../lib/impl/DistributedCacheHelper.java        |  52 ++
 .../lib/impl/FileOutputConfigurator.java        | 187 +++++
 .../mapreduce/lib/impl/InputConfigurator.java   | 796 +++++++++++++++++++
 .../mapreduce/lib/impl/OutputConfigurator.java  | 204 +++++
 .../client/mapreduce/lib/impl/package-info.java |  34 +
 .../lib/partition/RangePartitioner.java         |   2 +-
 .../mapreduce/lib/util/ConfiguratorBase.java    | 397 ---------
 .../lib/util/DistributedCacheHelper.java        |  40 -
 .../lib/util/FileOutputConfigurator.java        | 187 -----
 .../mapreduce/lib/util/InputConfigurator.java   | 795 ------------------
 .../mapreduce/lib/util/OutputConfigurator.java  | 204 -----
 .../client/mapreduce/lib/util/package-info.java |  34 -
 .../lib/impl/ConfiguratorBaseTest.java          | 129 +++
 .../lib/util/ConfiguratorBaseTest.java          | 152 ----
 27 files changed, 1820 insertions(+), 1940 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index dad62ca,edcfc53..5af78d2
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@@ -136,28 -136,28 +136,6 @@@ public abstract class AbstractInputForm
    }
  
    /**
--   * Gets the serialized token class from either the configuration or the token file.
--   * 
--   * @since 1.5.0
--   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
--   */
--  @Deprecated
--  protected static String getTokenClass(JobConf job) {
--    return getAuthenticationToken(job).getClass().getName();
--  }
--
--  /**
--   * Gets the serialized token from either the configuration or the token file.
--   * 
--   * @since 1.5.0
--   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
--   */
--  @Deprecated
--  protected static byte[] getToken(JobConf job) {
--    return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
--  }
--
--  /**
     * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
     * 
     * @param job
@@@ -176,23 -176,23 +154,6 @@@
     * 
     * @param job
     *          the Hadoop job instance to be configured
--   * @param instanceName
--   *          the Accumulo instance name
--   * @param zooKeepers
--   *          a comma-separated list of zookeeper servers
--   * @since 1.5.0
--   * @deprecated since 1.6.0; Use {@link #setZooKeeperInstance(JobConf, ClientConfiguration)} instead.
--   */
--  @Deprecated
--  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-     InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
-   }
- 
-   /**
-    * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-    *
-    * @param job
-    *          the Hadoop job instance to be configured
 -    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
     * @since 1.6.0
@@@ -221,7 -221,7 +182,7 @@@
     *          the Hadoop context for the configured job
     * @return an Accumulo instance
     * @since 1.5.0
--   * @see #setZooKeeperInstance(JobConf, String, String)
++   * @see #setZooKeeperInstance(JobConf, ClientConfiguration)
     * @see #setMockInstance(JobConf, String)
     */
    protected static Instance getInstance(JobConf job) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index 1ec4c41,04410c4..122b4cd
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@@ -176,24 -176,24 +176,6 @@@ public class AccumuloOutputFormat imple
     * 
     * @param job
     *          the Hadoop job instance to be configured
--   * @param instanceName
--   *          the Accumulo instance name
--   * @param zooKeepers
--   *          a comma-separated list of zookeeper servers
--   * @since 1.5.0
--   * @deprecated since 1.6.0; Use {@link #setZooKeeperInstance(JobConf, ClientConfiguration)} instead.
--   */
--
--  @Deprecated
--  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-     OutputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
-   }
- 
-   /**
-    * Configures a {@link ZooKeeperInstance} for this job.
-    * 
-    * @param job
-    *          the Hadoop job instance to be configured
 -    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.
     * @since 1.6.0
@@@ -222,7 -222,7 +204,7 @@@
     *          the Hadoop context for the configured job
     * @return an Accumulo instance
     * @since 1.5.0
--   * @see #setZooKeeperInstance(JobConf, String, String)
++   * @see #setZooKeeperInstance(JobConf, ClientConfiguration)
     * @see #setMockInstance(JobConf, String)
     */
    protected static Instance getInstance(JobConf job) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 53abbbe,19a674f..836cff9
--- 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
@@@ -180,23 -180,23 +180,6 @@@ public abstract class AbstractInputForm
     * 
     * @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) {
-     InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
-   }
- 
-   /**
-    * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-    *
-    * @param job
-    *          the Hadoop job instance to be configured
 -    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
     * @since 1.6.0
@@@ -225,7 -225,7 +208,7 @@@
     *          the Hadoop context for the configured job
     * @return an Accumulo instance
     * @since 1.5.0
--   * @see #setZooKeeperInstance(Job, String, String)
++   * @see #setZooKeeperInstance(Job, ClientConfiguration)
     * @see #setMockInstance(Job, String)
     */
    protected static Instance getInstance(JobContext context) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index 2c01b0d,b1ac2a7..af9bbae
--- 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
@@@ -177,23 -177,23 +177,6 @@@ public class AccumuloOutputFormat exten
     * 
     * @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) {
-     OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
-   }
- 
-   /**
-    * Configures a {@link ZooKeeperInstance} for this job.
-    * 
-    * @param job
-    *          the Hadoop job instance to be configured
 -    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.
     * @since 1.6.0

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
index 0000000,33ca5d2..c86623a
mode 000000,100644..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
@@@ -1,0 -1,372 +1,375 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.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 static com.google.common.base.Preconditions.checkArgument;
++
+ import java.io.IOException;
+ import java.net.URI;
+ import java.net.URISyntaxException;
++import java.nio.charset.StandardCharsets;
+ 
 -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;
+ 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;
+ 
+ /**
+  * @since 1.6.0
+  */
+ public class ConfiguratorBase {
+ 
+   /**
+    * Configuration keys for {@link Instance#getConnector(String, AuthenticationToken)}.
+    * 
+    * @since 1.6.0
+    */
+   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;
+     }
+   }
+ 
+   /**
+    * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link MockInstance}.
+    * 
+    * @since 1.6.0
+    */
+   public static enum InstanceOpts {
+     TYPE, NAME, ZOO_KEEPERS, CLIENT_CONFIG;
+   }
+ 
+   /**
+    * Configuration keys for general configuration options.
+    * 
+    * @since 1.6.0
+    */
+   public 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
+    * @since 1.6.0
+    */
+   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
+    * @since 1.6.0
+    */
+   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);
++    checkArgument(principal != null, "principal is null");
++    checkArgument(token != null, "token is null");
+     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
+     conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
+     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);
++    checkArgument(principal != null, "principal is null");
++    checkArgument(tokenFile != null, "tokenFile is null");
+ 
+     try {
+       DistributedCacheHelper.addCacheFile(new URI(tokenFile), conf);
+     } catch (URISyntaxException e) {
+       throw new IllegalStateException("Unable to add tokenFile \"" + tokenFile + "\" to distributed cache.");
+     }
+ 
+     conf.setBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), true);
+     conf.set(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL), principal);
+     conf.set(enumToConfKey(implementingClass, ConnectorInfo.TOKEN), TokenSource.FILE.prefix() + tokenFile);
+   }
+ 
+   /**
+    * Determines if the connector info has already been set for this instance.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return true if the connector info has already been set, false otherwise
+    * @since 1.6.0
+    * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
+    */
+   public static Boolean isConnectorInfoSet(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, ConnectorInfo.IS_CONFIGURED), false);
+   }
+ 
+   /**
+    * Gets the user name from the configuration.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return the principal
+    * @since 1.6.0
+    * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
+    */
+   public static String getPrincipal(Class<?> implementingClass, Configuration conf) {
+     return conf.get(enumToConfKey(implementingClass, ConnectorInfo.PRINCIPAL));
+   }
+ 
+   /**
+    * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return the principal's authentication token
+    * @since 1.6.0
+    * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
+    * @see #setConnectorInfo(Class, Configuration, String, String)
+    */
+   public static AuthenticationToken getAuthenticationToken(Class<?> implementingClass, Configuration conf) {
+     String token = conf.get(enumToConfKey(implementingClass, ConnectorInfo.TOKEN));
+     if (token == null || token.isEmpty())
+       return null;
+     if (token.startsWith(TokenSource.INLINE.prefix())) {
+       String[] args = token.substring(TokenSource.INLINE.prefix().length()).split(":", 2);
+       if (args.length == 2)
 -        return AuthenticationTokenSerializer.deserialize(args[0], Base64.decodeBase64(args[1].getBytes(Constants.UTF8)));
++        return AuthenticationTokenSerializer.deserialize(args[0], Base64.decodeBase64(args[1].getBytes(StandardCharsets.UTF_8)));
+     } 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");
+   }
+ 
+   /**
+    * Reads from the token file in distributed cache. Currently, the token file stores data separated by colons e.g. principal:token_class:token
+    * 
+    * @param conf
+    *          the Hadoop context for the configured job
+    * @return path to the token file as a String
+    * @since 1.6.0
+    * @see #setConnectorInfo(Class, Configuration, String, AuthenticationToken)
+    */
+   public static AuthenticationToken getTokenFromFile(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());
+     }
+   }
+ 
+   /**
+    * 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.6.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());
+     }
+   }
+ 
+   /**
+    * Configures a {@link 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
+    */
+   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);
++    checkArgument(instanceName != null, "instanceName is null");
+     conf.set(enumToConfKey(implementingClass, InstanceOpts.NAME), 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
+    * @since 1.6.0
+    * @see #setZooKeeperInstance(Class, Configuration, ClientConfiguration)
+    * @see #setMockInstance(Class, Configuration, String)
+    */
+   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);
+   }
+ 
+   /**
+    * Sets the log level for this job.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param level
+    *          the logging level
+    * @since 1.6.0
+    */
+   public static void setLogLevel(Class<?> implementingClass, Configuration conf, Level level) {
 -    ArgumentChecker.notNull(level);
++    checkArgument(level != null, "level is null");
+     Logger.getLogger(implementingClass).setLevel(level);
+     conf.setInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), level.toInt());
+   }
+ 
+   /**
+    * Gets the log level from this configuration.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return the log level
+    * @since 1.6.0
+    * @see #setLogLevel(Class, Configuration, Level)
+    */
+   public static Level getLogLevel(Class<?> implementingClass, Configuration conf) {
+     return Level.toLevel(conf.getInt(enumToConfKey(implementingClass, GeneralOpts.LOG_LEVEL), Level.INFO.toInt()));
+   }
+ 
+ }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index 0000000,cfd9aa2..7657c3c
mode 000000,100644..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
@@@ -1,0 -1,795 +1,796 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.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 static org.apache.accumulo.core.util.ArgumentChecker.notNull;
++import static com.google.common.base.Preconditions.checkArgument;
+ 
+ import java.io.ByteArrayInputStream;
+ import java.io.ByteArrayOutputStream;
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
++import java.nio.charset.StandardCharsets;
+ 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;
+ 
+ /**
+  * @since 1.6.0
+  */
+ public class InputConfigurator extends ConfiguratorBase {
+ 
+   /**
+    * Configuration keys for {@link Scanner}.
+    * 
+    * @since 1.6.0
+    */
+   public static enum ScanOpts {
+     TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS, TABLE_CONFIGS
+   }
+ 
+   /**
+    * Configuration keys for various features.
+    * 
+    * @since 1.6.0
+    */
+   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
+    * @since 1.6.0
+    */
+   public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
 -    notNull(tableName);
++    checkArgument(tableName != null, "tableName is null");
+     conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
+   }
+ 
+   /**
+    * Sets the name of the input table, over which this job will scan.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @since 1.6.0
+    */
+   public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
+     return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
+   }
+ 
+   /**
+    * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param auths
+    *          the user's authorizations
+    * @since 1.6.0
+    */
+   public static void setScanAuthorizations(Class<?> implementingClass, Configuration conf, Authorizations auths) {
+     if (auths != null && !auths.isEmpty())
+       conf.set(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS), auths.serialize());
+   }
+ 
+   /**
+    * Gets the authorizations to set for the scans from the configuration.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return the Accumulo scan authorizations
+    * @since 1.6.0
+    * @see #setScanAuthorizations(Class, Configuration, Authorizations)
+    */
+   public static Authorizations getScanAuthorizations(Class<?> implementingClass, Configuration conf) {
+     String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS));
 -    return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes(Constants.UTF8));
++    return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes(StandardCharsets.UTF_8));
+   }
+ 
+   /**
+    * Sets the input ranges to scan on all input tables for this job. If not set, the entire table will be scanned.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param ranges
+    *          the ranges that will be mapped over
+    * @throws IllegalArgumentException
+    *           if the ranges cannot be encoded into base 64
+    * @since 1.6.0
+    */
+   public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
 -    notNull(ranges);
++    checkArgument(ranges != null, "ranges is null");
+ 
+     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);
+     }
+   }
+ 
+   /**
+    * Gets the ranges to scan over from a job.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return the ranges
+    * @throws IOException
+    *           if the ranges have been encoded improperly
+    * @since 1.6.0
+    * @see #setRanges(Class, Configuration, Collection)
+    */
+   public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
+ 
+     Collection<String> encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
+     List<Range> ranges = new ArrayList<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;
+   }
+ 
+   /**
+    * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return a list of iterators
+    * @since 1.6.0
+    * @see #addIterator(Class, Configuration, IteratorSetting)
+    */
+   public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
+     String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
+ 
+     // If no iterators are present, return an empty list
+     if (iterators == null || iterators.isEmpty())
+       return new ArrayList<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.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param columnFamilyColumnQualifierPairs
+    *          a pair of {@link Text} objects corresponding to column family and column qualifier. If the column qualifier is null, the entire column family is
+    *          selected. An empty set is the default and is equivalent to scanning the all columns.
+    * @throws IllegalArgumentException
+    *           if the column family is null
+    * @since 1.6.0
+    */
+   public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 -    notNull(columnFamilyColumnQualifierPairs);
++    checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null");
+     String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+     conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
+   }
+ 
+   public static String[] serializeColumns(Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 -    notNull(columnFamilyColumnQualifierPairs);
++    checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null");
+     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);
++      String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), StandardCharsets.UTF_8);
+       if (column.getSecond() != null)
 -        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), Constants.UTF8);
++        col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), StandardCharsets.UTF_8);
+       columnStrings.add(col);
+     }
+ 
+     return columnStrings.toArray(new String[0]);
+   }
+ 
+   /**
+    * Gets the columns to be mapped over from this job.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return a set of columns
+    * @since 1.6.0
+    * @see #fetchColumns(Class, Configuration, Collection)
+    */
+   public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
 -    notNull(conf);
++    checkArgument(conf != null, "conf is null");
+     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);
+   }
+ 
+   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)));
++      Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(StandardCharsets.UTF_8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(
++          StandardCharsets.UTF_8)));
++      Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes(StandardCharsets.UTF_8)));
+       columns.add(new Pair<Text,Text>(cf, cq));
+     }
+     return columns;
+   }
+ 
+   /**
+    * Encode an iterator on the input for the single input table associated with this job.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param cfg
+    *          the configuration of the iterator
+    * @throws IllegalArgumentException
+    *           if the iterator can't be serialized into the configuration
+    * @since 1.6.0
+    */
+   public static void addIterator(Class<?> implementingClass, Configuration conf, IteratorSetting cfg) {
+     ByteArrayOutputStream baos = new ByteArrayOutputStream();
+     String newIter;
+     try {
+       cfg.write(new DataOutputStream(baos));
 -      newIter = new String(Base64.encodeBase64(baos.toByteArray()), Constants.UTF8);
++      newIter = new String(Base64.encodeBase64(baos.toByteArray()), StandardCharsets.UTF_8);
+       baos.close();
+     } catch (IOException e) {
+       throw new IllegalArgumentException("unable to serialize IteratorSetting");
+     }
+ 
+     String confKey = enumToConfKey(implementingClass, ScanOpts.ITERATORS);
+     String iterators = conf.get(confKey);
+     // No iterators specified yet, create a new string
+     if (iterators == null || iterators.isEmpty()) {
+       iterators = newIter;
+     } else {
+       // append the next iterator & reset
+       iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
+     }
+     // Store the iterators w/ the job
+     conf.set(confKey, iterators);
+   }
+ 
+   /**
+    * Controls the automatic adjustment of ranges for this job. This feature merges overlapping ranges, then splits them to align with tablet boundaries.
+    * Disabling this feature will cause exactly one Map task to be created for each specified range. The default setting is enabled. *
+    * 
+    * <p>
+    * By default, this feature is <b>enabled</b>.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param enableFeature
+    *          the feature is enabled if true, disabled otherwise
+    * @see #setRanges(Class, Configuration, Collection)
+    * @since 1.6.0
+    */
+   public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+     conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
+   }
+ 
+   /**
+    * Determines whether a configuration has auto-adjust ranges enabled.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return false if the feature is disabled, true otherwise
+    * @since 1.6.0
+    * @see #setAutoAdjustRanges(Class, Configuration, boolean)
+    */
+   public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
+   }
+ 
+   /**
+    * Controls the use of the {@link IsolatedScanner} in this job.
+    * 
+    * <p>
+    * By default, this feature is <b>disabled</b>.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param enableFeature
+    *          the feature is enabled if true, disabled otherwise
+    * @since 1.6.0
+    */
+   public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+     conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
+   }
+ 
+   /**
+    * Determines whether a configuration has isolation enabled.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return true if the feature is enabled, false otherwise
+    * @since 1.6.0
+    * @see #setScanIsolation(Class, Configuration, boolean)
+    */
+   public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
+   }
+ 
+   /**
+    * Controls the use of the {@link ClientSideIteratorScanner} in this job. Enabling this feature will cause the iterator stack to be constructed within the Map
+    * task, rather than within the Accumulo TServer. To use this feature, all classes needed for those iterators must be available on the classpath for the task.
+    * 
+    * <p>
+    * By default, this feature is <b>disabled</b>.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param enableFeature
+    *          the feature is enabled if true, disabled otherwise
+    * @since 1.6.0
+    */
+   public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+     conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
+   }
+ 
+   /**
+    * Determines whether a configuration uses local iterators.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return true if the feature is enabled, false otherwise
+    * @since 1.6.0
+    * @see #setLocalIterators(Class, Configuration, boolean)
+    */
+   public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
+   }
+ 
+   /**
+    * <p>
+    * Enable reading offline tables. By default, this feature is disabled and only online tables are scanned. This will make the map reduce job directly read the
+    * table's files. If the table is not offline, then the job will fail. If the table comes online during the map reduce job, it is likely that the job will
+    * fail.
+    * 
+    * <p>
+    * To use this option, the map reduce user will need access to read the Accumulo directory in HDFS.
+    * 
+    * <p>
+    * Reading the offline table will create the scan time iterator stack in the map process. So any iterators that are configured for the table will need to be
+    * on the mapper's classpath.
+    * 
+    * <p>
+    * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
+    * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
+    * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
+    * 
+    * <p>
+    * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
+    * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
+    * 
+    * <p>
+    * By default, this feature is <b>disabled</b>.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param enableFeature
+    *          the feature is enabled if true, disabled otherwise
+    * @since 1.6.0
+    */
+   public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+     conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
+   }
+ 
+   /**
+    * Determines whether a configuration has the offline table scan feature enabled.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return true if the feature is enabled, false otherwise
+    * @since 1.6.0
+    * @see #setOfflineTableScan(Class, Configuration, boolean)
+    */
+   public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
+   }
+ 
+   /**
+    * 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);
+   }
+ 
+   /**
+    * Initializes an Accumulo {@link TabletLocator} based on the configuration.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param tableId
+    *          The table id for which to initialize the {@link TabletLocator}
+    * @return an Accumulo tablet locator
+    * @throws TableNotFoundException
+    *           if the table name set on the configuration doesn't exist
+    * @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 new MockTabletLocator();
+     Instance instance = getInstance(implementingClass, conf);
+     return TabletLocator.getLocator(instance, new Text(tableId));
+   }
+ 
+   // 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
+    * @since 1.6.0
+    */
+   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.
+    * 
+    * @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;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/OutputConfigurator.java
index 0000000,0ba42cd..727971a
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,204 +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 java.nio.charset.StandardCharsets;
+ 
 -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);
++      serialized = new String(baos.toByteArray(), StandardCharsets.UTF_8);
+       baos.close();
+     } catch (IOException e) {
+       throw new IllegalArgumentException("unable to serialize " + BatchWriterConfig.class.getName());
+     }
+     conf.set(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG), serialized);
+   }
+ 
+   /**
+    * Gets the {@link BatchWriterConfig} settings.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return the configuration object
+    * @since 1.6.0
+    * @see #setBatchWriterOptions(Class, Configuration, BatchWriterConfig)
+    */
+   public static BatchWriterConfig getBatchWriterOptions(Class<?> implementingClass, Configuration conf) {
+     String serialized = conf.get(enumToConfKey(implementingClass, WriteOpts.BATCH_WRITER_CONFIG));
+     BatchWriterConfig bwConfig = new BatchWriterConfig();
+     if (serialized == null || serialized.isEmpty()) {
+       return bwConfig;
+     } else {
+       try {
 -        ByteArrayInputStream bais = new ByteArrayInputStream(serialized.getBytes(Constants.UTF8));
++        ByteArrayInputStream bais = new ByteArrayInputStream(serialized.getBytes(StandardCharsets.UTF_8));
+         bwConfig.readFields(new DataInputStream(bais));
+         bais.close();
+         return bwConfig;
+       } catch (IOException e) {
+         throw new IllegalArgumentException("unable to serialize " + BatchWriterConfig.class.getName());
+       }
+     }
+   }
+ 
+   /**
+    * Sets the directive to create new tables, as necessary. Table names can only be alpha-numeric and underscores.
+    * 
+    * <p>
+    * By default, this feature is <b>disabled</b>.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param enableFeature
+    *          the feature is enabled if true, disabled otherwise
+    * @since 1.6.0
+    */
+   public static void setCreateTables(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+     conf.setBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), enableFeature);
+   }
+ 
+   /**
+    * Determines whether tables are permitted to be created as needed.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return true if the feature is disabled, false otherwise
+    * @since 1.6.0
+    * @see #setCreateTables(Class, Configuration, boolean)
+    */
+   public static Boolean canCreateTables(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, Features.CAN_CREATE_TABLES), false);
+   }
+ 
+   /**
+    * Sets the directive to use simulation mode for this job. In simulation mode, no output is produced. This is useful for testing.
+    * 
+    * <p>
+    * By default, this feature is <b>disabled</b>.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @param enableFeature
+    *          the feature is enabled if true, disabled otherwise
+    * @since 1.6.0
+    */
+   public static void setSimulationMode(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
+     conf.setBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), enableFeature);
+   }
+ 
+   /**
+    * Determines whether this feature is enabled.
+    * 
+    * @param implementingClass
+    *          the class whose name will be used as a prefix for the property configuration key
+    * @param conf
+    *          the Hadoop configuration object to configure
+    * @return true if the feature is enabled, false otherwise
+    * @since 1.6.0
+    * @see #setSimulationMode(Class, Configuration, boolean)
+    */
+   public static Boolean getSimulationMode(Class<?> implementingClass, Configuration conf) {
+     return conf.getBoolean(enumToConfKey(implementingClass, Features.SIMULATION_MODE), false);
+   }
+ 
+ }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf102d07/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java
index 0a10871,54730ef..1b7501c
--- 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 -26,8 +27,7 @@@ import java.util.Arrays
  import java.util.Scanner;
  import java.util.TreeSet;
  
- import org.apache.accumulo.core.client.mapreduce.lib.util.DistributedCacheHelper;
 -import org.apache.accumulo.core.Constants;
+ 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;