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:21 UTC

[3/4] git commit: ACCUMULO-2659 Properly deprecate public API changes to mapred utils

ACCUMULO-2659 Properly deprecate public API changes to mapred utils


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 382bfdcef9f34685724b3d9614f0df7c1cda26b8
Parents: 8f57280
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Apr 11 15:18:38 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Apr 11 15:20:39 2014 -0400

----------------------------------------------------------------------
 .../core/client/mapred/AbstractInputFormat.java |  27 +-
 .../client/mapred/AccumuloFileOutputFormat.java |   2 +-
 .../mapred/AccumuloMultiTableInputFormat.java   |   2 +-
 .../client/mapred/AccumuloOutputFormat.java     |   4 +-
 .../core/client/mapred/InputFormatBase.java     |   2 +-
 .../client/mapreduce/AbstractInputFormat.java   |  30 +-
 .../mapreduce/AccumuloFileOutputFormat.java     |   2 +-
 .../AccumuloMultiTableInputFormat.java          |   2 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |   4 +-
 .../core/client/mapreduce/InputFormatBase.java  |   2 +-
 .../core/client/mapreduce/RangeInputSplit.java  |   4 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java    | 372 +++++++++
 .../lib/impl/DistributedCacheHelper.java        |  52 ++
 .../lib/impl/FileOutputConfigurator.java        | 187 +++++
 .../mapreduce/lib/impl/InputConfigurator.java   | 795 +++++++++++++++++++
 .../mapreduce/lib/impl/OutputConfigurator.java  | 204 +++++
 .../client/mapreduce/lib/impl/package-info.java |  34 +
 .../lib/partition/RangePartitioner.java         |   2 +-
 .../mapreduce/lib/util/ConfiguratorBase.java    | 248 ++----
 .../lib/util/DistributedCacheHelper.java        |  40 -
 .../lib/util/FileOutputConfigurator.java        |  83 +-
 .../mapreduce/lib/util/InputConfigurator.java   | 531 +++----------
 .../mapreduce/lib/util/OutputConfigurator.java  |  90 +--
 .../client/mapreduce/lib/util/package-info.java |  15 +-
 .../lib/impl/ConfiguratorBaseTest.java          | 129 +++
 .../lib/util/ConfiguratorBaseTest.java          | 152 ----
 26 files changed, 2054 insertions(+), 961 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index dad62ca..edcfc53 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -44,7 +44,7 @@ import org.apache.accumulo.core.client.impl.ScannerImpl;
 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.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.Key;
@@ -185,12 +185,12 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
    */
   @Deprecated
   public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
+    setZooKeeperInstance(job, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
    * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
+   * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param clientConfig
@@ -372,7 +372,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
-      
+
       Instance instance = split.getInstance();
       if (null == instance) {
         instance = getInstance(job);
@@ -398,7 +398,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
       // in case the table name changed, we can still use the previous name for terms of configuration,
       // but the scanner will use the table id resolved at job setup time
       InputTableConfig tableConfig = getInputTableConfig(job, split.getTableName());
-      
+
       Boolean isOffline = split.isOffline();
       if (null == isOffline) {
         isOffline = tableConfig.isOfflineScan();
@@ -413,12 +413,12 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
       if (null == usesLocalIterators) {
         usesLocalIterators = tableConfig.shouldUseLocalIterators();
       }
-      
+
       List<IteratorSetting> iterators = split.getIterators();
       if (null == iterators) {
         iterators = tableConfig.getIterators();
       }
-      
+
       Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
       if (null == columns) {
         columns = tableConfig.getFetchedColumns();
@@ -448,7 +448,6 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
         throw new IOException(e);
       }
 
-      
       // setup a scanner within the bounds of this split
       for (Pair<Text,Text> c : columns) {
         if (c.getSecond() != null) {
@@ -511,7 +510,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
     for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
       String tableName = tableConfigEntry.getKey();
       InputTableConfig tableConfig = tableConfigEntry.getValue();
-      
+
       Instance instance = getInstance(job);
       boolean mockInstance;
       String tableId;
@@ -527,11 +526,11 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
         }
         mockInstance = false;
       }
-      
+
       Authorizations auths = getScanAuthorizations(job);
       String principal = getPrincipal(job);
       AuthenticationToken token = getAuthenticationToken(job);
-      
+
       boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
       List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
       if (ranges.isEmpty()) {
@@ -593,7 +592,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
             if (autoAdjust) {
               // divide ranges into smaller ranges, based on the tablets
               RangeInputSplit split = new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location});
-              
+
               split.setOffline(tableConfig.isOfflineScan());
               split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
               split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
@@ -606,7 +605,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
               split.setAuths(auths);
               split.setIterators(tableConfig.getIterators());
               split.setLogLevel(logLevel);
-              
+
               splits.add(split);
             } else {
               // don't divide ranges
@@ -636,7 +635,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
           split.setAuths(auths);
           split.setIterators(tableConfig.getIterators());
           split.setLogLevel(logLevel);
-          
+
           splits.add(split);
         }
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
index badcd83..8a1d6df 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormat.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.client.mapred;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.accumulo.core.client.mapreduce.lib.util.FileOutputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ArrayByteSequence;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
index 2ef9931..bbafef5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
@@ -21,7 +21,7 @@ import java.util.Map;
 
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.format.DefaultFormatter;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index 1ec4c41..04410c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -34,7 +34,7 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mapreduce.lib.util.OutputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -186,7 +186,7 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
 
   @Deprecated
   public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
+    setZooKeeperInstance(job, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
index f2ac488..332e1f1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 53abbbe..19a674f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -44,7 +44,7 @@ import org.apache.accumulo.core.client.impl.OfflineScanner;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.Key;
@@ -189,12 +189,12 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
    */
   @Deprecated
   public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
+    setZooKeeperInstance(job, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**
    * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
-   *
+   * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param clientConfig
@@ -381,7 +381,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
-      
+
       Instance instance = split.getInstance();
       if (null == instance) {
         instance = getInstance(attempt);
@@ -407,7 +407,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       // in case the table name changed, we can still use the previous name for terms of configuration,
       // but the scanner will use the table id resolved at job setup time
       InputTableConfig tableConfig = getInputTableConfig(attempt, split.getTableName());
-      
+
       Boolean isOffline = split.isOffline();
       if (null == isOffline) {
         isOffline = tableConfig.isOfflineScan();
@@ -422,12 +422,12 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       if (null == usesLocalIterators) {
         usesLocalIterators = tableConfig.shouldUseLocalIterators();
       }
-      
+
       List<IteratorSetting> iterators = split.getIterators();
       if (null == iterators) {
         iterators = tableConfig.getIterators();
       }
-      
+
       Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
       if (null == columns) {
         columns = tableConfig.getFetchedColumns();
@@ -452,7 +452,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        
+
         setupIterators(attempt, scanner, split.getTableName(), split);
       } catch (Exception e) {
         throw new IOException(e);
@@ -490,12 +490,12 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
      * The Key that should be returned to the client
      */
     protected K currentK = null;
-    
+
     /**
      * The Value that should be return to the client
      */
     protected V currentV = null;
-    
+
     /**
      * The Key that is used to determine progress in the current InputSplit. It is not returned to the client and is only used internally
      */
@@ -542,7 +542,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
 
       String tableName = tableConfigEntry.getKey();
       InputTableConfig tableConfig = tableConfigEntry.getValue();
-      
+
       Instance instance = getInstance(context);
       boolean mockInstance;
       String tableId;
@@ -558,7 +558,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
         }
         mockInstance = false;
       }
-      
+
       Authorizations auths = getScanAuthorizations(context);
       String principal = getPrincipal(context);
       AuthenticationToken token = getAuthenticationToken(context);
@@ -625,7 +625,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
             if (autoAdjust) {
               // divide ranges into smaller ranges, based on the tablets
               RangeInputSplit split = new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location});
-              
+
               split.setOffline(tableConfig.isOfflineScan());
               split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
               split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
@@ -638,7 +638,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
               split.setAuths(auths);
               split.setIterators(tableConfig.getIterators());
               split.setLogLevel(logLevel);
-              
+
               splits.add(split);
             } else {
               // don't divide ranges
@@ -668,7 +668,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
           split.setAuths(auths);
           split.setIterators(tableConfig.getIterators());
           split.setLogLevel(logLevel);
-          
+
           splits.add(split);
         }
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
index d78219c..196fb04 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormat.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.core.client.mapreduce;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.accumulo.core.client.mapreduce.lib.util.FileOutputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ArrayByteSequence;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index c514385..af1001f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index 2c01b0d..b1ac2a7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -34,7 +34,7 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mapreduce.lib.util.OutputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -186,7 +186,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    */
   @Deprecated
   public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
-    OutputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
+    setZooKeeperInstance(job, new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index a934d71..70dad6c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 85c22fa..47b34e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -32,8 +32,8 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mapreduce.lib.util.ConfiguratorBase.TokenSource;
-import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.TokenSource;
 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;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
new file mode 100644
index 0000000..33ca5d2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
@@ -0,0 +1,372 @@
+/*
+ * 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.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;
+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);
+    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);
+  }
+
+  /**
+   * 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)));
+    } 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);
+    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);
+    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/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/DistributedCacheHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/DistributedCacheHelper.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/DistributedCacheHelper.java
new file mode 100644
index 0000000..c694b9a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/DistributedCacheHelper.java
@@ -0,0 +1,52 @@
+/*
+ * 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.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * @since 1.6.0
+ */
+@SuppressWarnings("deprecation")
+public class DistributedCacheHelper {
+
+  /**
+   * @since 1.6.0
+   */
+  public static void addCacheFile(URI uri, Configuration conf) {
+    DistributedCache.addCacheFile(uri, conf);
+  }
+
+  /**
+   * @since 1.6.0
+   */
+  public static URI[] getCacheFiles(Configuration conf) throws IOException {
+    return DistributedCache.getCacheFiles(conf);
+  }
+
+  /**
+   * @since 1.6.0
+   */
+  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/impl/FileOutputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
new file mode 100644
index 0000000..ce84209
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/FileOutputConfigurator.java
@@ -0,0 +1,187 @@
+/*
+ * 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.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;
+
+/**
+ * @since 1.6.0
+ */
+public class FileOutputConfigurator extends ConfiguratorBase {
+
+  /**
+   * Configuration keys for {@link AccumuloConfiguration}.
+   * 
+   * @since 1.6.0
+   */
+  public static enum Opts {
+    ACCUMULO_PROPERTIES;
+  }
+
+  /**
+   * The supported Accumulo properties we set in this OutputFormat, that change the behavior of the RecordWriter.<br />
+   * These properties correspond to the supported public static setter methods available to this class.
+   * 
+   * @param property
+   *          the Accumulo property to check
+   * @since 1.6.0
+   */
+  protected static Boolean isSupportedAccumuloProperty(Property property) {
+    switch (property) {
+      case TABLE_FILE_COMPRESSION_TYPE:
+      case TABLE_FILE_COMPRESSED_BLOCK_SIZE:
+      case TABLE_FILE_BLOCK_SIZE:
+      case TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX:
+      case TABLE_FILE_REPLICATION:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Helper for transforming Accumulo configuration properties into something that can be stored safely inside the Hadoop Job configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param property
+   *          the supported Accumulo property
+   * @param value
+   *          the value of the property to set
+   * @since 1.6.0
+   */
+  private static <T> void setAccumuloProperty(Class<?> implementingClass, Configuration conf, Property property, T value) {
+    if (isSupportedAccumuloProperty(property)) {
+      String val = String.valueOf(value);
+      if (property.getType().isValidFormat(val))
+        conf.set(enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + "." + property.getKey(), val);
+      else
+        throw new IllegalArgumentException("Value is not appropriate for property type '" + property.getType() + "'");
+    } else
+      throw new IllegalArgumentException("Unsupported configuration property " + property.getKey());
+  }
+
+  /**
+   * This helper method provides an AccumuloConfiguration object constructed from the Accumulo defaults, and overridden with Accumulo properties that have been
+   * stored in the Job's configuration.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @since 1.6.0
+   */
+  public static AccumuloConfiguration getAccumuloConfiguration(Class<?> implementingClass, Configuration conf) {
+    String prefix = enumToConfKey(implementingClass, Opts.ACCUMULO_PROPERTIES) + ".";
+    ConfigurationCopy acuConf = new ConfigurationCopy(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;
+  }
+
+  /**
+   * Sets the compression type to use for data blocks. Specifying a compression may require additional libraries to be available to your Job.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param compressionType
+   *          one of "none", "gz", "lzo", or "snappy"
+   * @since 1.6.0
+   */
+  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);
+  }
+
+  /**
+   * Sets the size for data blocks within each file.<br />
+   * Data blocks are a span of key/value pairs stored in the file that are compressed and indexed as a group.
+   * 
+   * <p>
+   * Making this value smaller may increase seek performance, but at the cost of increasing the size of the indexes (which can also affect seek performance).
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param dataBlockSize
+   *          the block size, in bytes
+   * @since 1.6.0
+   */
+  public static void setDataBlockSize(Class<?> implementingClass, Configuration conf, long dataBlockSize) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, dataBlockSize);
+  }
+
+  /**
+   * Sets the size for file blocks in the file system; file blocks are managed, and replicated, by the underlying file system.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param fileBlockSize
+   *          the block size, in bytes
+   * @since 1.6.0
+   */
+  public static void setFileBlockSize(Class<?> implementingClass, Configuration conf, long fileBlockSize) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_BLOCK_SIZE, fileBlockSize);
+  }
+
+  /**
+   * Sets the size for index blocks within each file; smaller blocks means a deeper index hierarchy within the file, while larger blocks mean a more shallow
+   * index hierarchy within the file. This can affect the performance of queries.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param indexBlockSize
+   *          the block size, in bytes
+   * @since 1.6.0
+   */
+  public static void setIndexBlockSize(Class<?> implementingClass, Configuration conf, long indexBlockSize) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, indexBlockSize);
+  }
+
+  /**
+   * Sets the file system replication factor for the resulting file, overriding the file system default.
+   * 
+   * @param implementingClass
+   *          the class whose name will be used as a prefix for the property configuration key
+   * @param conf
+   *          the Hadoop configuration object to configure
+   * @param replication
+   *          the number of replicas for produced files
+   * @since 1.6.0
+   */
+  public static void setReplication(Class<?> implementingClass, Configuration conf, int replication) {
+    setAccumuloProperty(implementingClass, conf, Property.TABLE_FILE_REPLICATION, replication);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/382bfdce/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
new file mode 100644
index 0000000..cfd9aa2
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -0,0 +1,795 @@
+/*
+ * 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 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;
+
+/**
+ * @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);
+    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));
+  }
+
+  /**
+   * 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);
+
+    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);
+    String[] columnStrings = serializeColumns(columnFamilyColumnQualifierPairs);
+    conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), columnStrings);
+  }
+
+  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]);
+  }
+
+  /**
+   * 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);
+    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)));
+      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);
+      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;
+  }
+}