You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cj...@apache.org on 2013/10/02 05:22:40 UTC

[04/11] git commit: The original single-table setters/getters now populate a "default" TableQueryConfig object under the hood. This should make the switch over much easier. Deprecated single table methods in light of the API changes for the new configura

The original single-table setters/getters now populate a "default" TableQueryConfig object under the hood. This should make the switch over much easier. Deprecated single table methods in light of the API changes for the new configuration object. ACCUMULO-391


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

Branch: refs/heads/ACCUMULO-391
Commit: e6a7c962f707487d832ba4b16c1f9066d13ff8f1
Parents: fdf4cad
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Sat Sep 28 22:53:42 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 1 21:46:16 2013 -0400

----------------------------------------------------------------------
 .../core/client/mapred/InputFormatBase.java     |  17 +-
 .../core/client/mapreduce/InputFormatBase.java  | 392 ++++++++-----------
 .../mapreduce/lib/util/InputConfigurator.java   | 147 +++++--
 .../accumulo/core/conf/TableQueryConfig.java    |  22 +-
 .../mapreduce/AccumuloInputFormatTest.java      | 128 +-----
 5 files changed, 312 insertions(+), 394 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e6a7c962/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 ea6bfbc..2d4eadf 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
@@ -276,11 +276,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @see #setInputTableName(JobConf, String)
    */
   protected static String getInputTableName(JobConf job) {
-    String[] tableNames = InputConfigurator.getInputTableNames(CLASS, job);
-    if (tableNames.length > 0)
-      return tableNames[0];
-    else
-      return null;
+    return InputConfigurator.getInputTableName(CLASS, job);
   }
 
   /**
@@ -334,12 +330,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @see #setRanges(JobConf, Collection)
    */
   protected static List<Range> getRanges(JobConf job) throws IOException {
-    Map<String, List<Range>> tableRanges = InputConfigurator.getRanges(CLASS,job);
-    List<Range> ranges = tableRanges.get(getInputTableName(job));
-    if(ranges != null)
-      return ranges;
-    else
-      return new LinkedList<Range>();
+    return InputConfigurator.getRanges(CLASS,job);
   }
 
   /**
@@ -366,7 +357,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @see #fetchColumns(JobConf, Collection)
    */
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
-    return InputConfigurator.getFetchedColumns(CLASS, job, getInputTableName(job));
+    return InputConfigurator.getFetchedColumns(CLASS, job);
   }
 
   /**
@@ -392,7 +383,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @see #addIterator(JobConf, IteratorSetting)
    */
   protected static List<IteratorSetting> getIterators(JobConf job) {
-    return InputConfigurator.getDefaultIterators(CLASS,job);
+    return InputConfigurator.getIterators(CLASS,job);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e6a7c962/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 04ff718..d5208b4 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
@@ -91,10 +91,10 @@ import org.apache.log4j.Logger;
  * See {@link AccumuloInputFormat} for an example implementation.
  */
 public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
-
+  
   private static final Class<?> CLASS = AccumuloInputFormat.class;
   protected static final Logger log = Logger.getLogger(CLASS);
-
+  
   /**
    * Sets the connector information needed to communicate with Accumulo in this job.
    * 
@@ -114,7 +114,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setConnectorInfo(Job job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
     InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
   }
-
+  
   /**
    * Sets the connector information needed to communicate with Accumulo in this job.
    * 
@@ -133,7 +133,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setConnectorInfo(Job job, String principal, String tokenFile) throws AccumuloSecurityException {
     InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
   }
-
+  
   /**
    * Determines if the connector has been configured.
    * 
@@ -146,7 +146,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static Boolean isConnectorInfoSet(JobContext context) {
     return InputConfigurator.isConnectorInfoSet(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * Gets the user name from the configuration.
    * 
@@ -159,7 +159,20 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getPrincipal(JobContext context) {
     return InputConfigurator.getPrincipal(CLASS, getConfiguration(context));
   }
-
+  
+  /**
+   * Gets the table name from the configuration.
+   * 
+   * @param context
+   *          the Hadoop context for the configured job
+   * @return the table name
+   * @since 1.5.0
+   * @see #setInputTableName(Job, String)
+   */
+  protected static String getInputTableName(JobContext context) {
+    return InputConfigurator.getInputTableName(CLASS, getConfiguration(context));
+  }
+  
   /**
    * Gets the serialized token class from either the configuration or the token file.
    * 
@@ -170,7 +183,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static String getTokenClass(JobContext context) {
     return getAuthenticationToken(context).getClass().getName();
   }
-
+  
   /**
    * Gets the serialized token from either the configuration or the token file.
    * 
@@ -181,7 +194,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static byte[] getToken(JobContext context) {
     return AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
   }
-
+  
   /**
    * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
    * 
@@ -193,9 +206,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #setConnectorInfo(Job, String, String)
    */
   protected static AuthenticationToken getAuthenticationToken(JobContext context) {
-    return InputConfigurator.getAuthenticationToken(CLASS,getConfiguration(context));
+    return InputConfigurator.getAuthenticationToken(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * Configures a {@link ZooKeeperInstance} for this job.
    * 
@@ -210,7 +223,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
     InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
   }
-
+  
   /**
    * Configures a {@link MockInstance} for this job.
    * 
@@ -223,7 +236,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setMockInstance(Job job, String instanceName) {
     InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
   }
-
+  
   /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    * 
@@ -235,9 +248,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #setMockInstance(Job, String)
    */
   protected static Instance getInstance(JobContext context) {
-    return InputConfigurator.getInstance(CLASS,getConfiguration(context));
+    return InputConfigurator.getInstance(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * Sets the log level for this job.
    * 
@@ -248,9 +261,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @since 1.5.0
    */
   public static void setLogLevel(Job job, Level level) {
-    InputConfigurator.setLogLevel(CLASS,job.getConfiguration(),level);
+    InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
   }
-
+  
   /**
    * Gets the log level from this configuration.
    * 
@@ -263,10 +276,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static Level getLogLevel(JobContext context) {
     return InputConfigurator.getLogLevel(CLASS, getConfiguration(context));
   }
-
+  
   /**
-   * Sets the name of the input table, over which this job will scan. This method has been deprecated in favor of
-   * {@link InputFormatBase#setInputTableNames(org.apache.hadoop.mapreduce.Job, java.util.Collection)}
+   * Sets the name of the input table, over which this job will scan.
    * 
    * @param job
    *          the Hadoop job instance to be configured
@@ -276,26 +288,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   @Deprecated
   public static void setInputTableName(Job job, String tableName) {
-    InputConfigurator.setInputTableName(CLASS,job.getConfiguration(),tableName);
+    InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
   }
-
-  /**
-   * Sets the names of the input tables over which this job will scan.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param tableNames
-   *          the table to use when the tablename is null in the write call
-   * @since 1.6.0
-   */
-  @Deprecated
-  public static void setInputTableNames(Job job, Collection<String> tableNames) {
-    InputConfigurator.setInputTableNames(CLASS,job.getConfiguration(),tableNames);
-  }
-
+  
   /**
    * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
-   *
+   * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param auths
@@ -303,12 +301,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @since 1.5.0
    */
   public static void setScanAuthorizations(Job job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS,job.getConfiguration(),auths);
+    InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
   }
-
+  
   /**
    * Gets the authorizations to set for the scans from the configuration.
-   *
+   * 
    * @param context
    *          the Hadoop context for the configured job
    * @return the Accumulo scan authorizations
@@ -316,13 +314,13 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @see #setScanAuthorizations(Job, Authorizations)
    */
   protected static Authorizations getScanAuthorizations(JobContext context) {
-    return InputConfigurator.getScanAuthorizations(CLASS,getConfiguration(context));
+    return InputConfigurator.getScanAuthorizations(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * Sets the input ranges to scan for all tables associated with this job. This will be added to any per-table ranges that have been set using
-   * {@link #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Map)}
-   *
+   * {@link #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Collection)}
+   * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param ranges
@@ -331,43 +329,26 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   @Deprecated
   public static void setRanges(Job job, Collection<Range> ranges) {
-    InputConfigurator.setRanges(CLASS,job.getConfiguration(),ranges);
-  }
-
-  /**
-   * Sets the input ranges to scan per-table for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param ranges
-   *          the per-table ranges that will be mapped over
-   * @since 1.6.0
-   */
-  @Deprecated
-  public static void setRanges(Job job, Map<String,Collection<Range>> ranges) {
     InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
   }
-
+  
   /**
    * Gets the ranges to scan over from a job.
    * 
    * @param context
    *          the Hadoop context for the configured job
    * @return the ranges
-   * @throws IOException
-   *           if the ranges have been encoded improperly
    * @since 1.5.0
    * @see #setRanges(Job, Collection)
-   * @see #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Map)
+   * @see #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Collection)
    */
   @Deprecated
-  protected static Map<String,List<Range>> getRanges(JobContext context) throws IOException {
+  protected static List<Range> getRanges(JobContext context) throws IOException {
     return InputConfigurator.getRanges(CLASS, getConfiguration(context));
   }
-
+  
   /**
-   * Restricts the columns that will be mapped over for this job for all tables. These columns will be added to any per-table columns set with
-   * {@link #fetchColumns(org.apache.hadoop.mapreduce.Job, java.util.Map)}.
+   * Restricts the columns that will be mapped over for this job for the default input table.
    * 
    * @param job
    *          the Hadoop job instance to be configured
@@ -380,38 +361,20 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void fetchColumns(Job job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
   }
-
-  /**
-   * Restricts the columns that will be mapped over for this job per table..
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param columnFamilyColumnQualifierPairs
-   *          A map keyed by table name where the value is 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.
-   * @since 1.6.0
-   */
-  @Deprecated
-  public static void fetchColumns(Job job, Map<String,Collection<Pair<Text,Text>>> columnFamilyColumnQualifierPairs) {
-    InputConfigurator.fetchColumns(CLASS,job.getConfiguration(),columnFamilyColumnQualifierPairs);
-  }
-
+  
   /**
-   * Gets the columns to be mapped over from this job. Any default columns as well as per-table columns will be returned.
+   * Gets the columns to be mapped over from this job.
    * 
    * @param context
    *          the Hadoop context for the configured job
-   * @param table
-   *          the table for which to return the columns
    * @return a set of columns
-   * @since 1.6.0
+   * @since 1.5.0
    * @see #fetchColumns(Job, Collection)
    */
-  @Deprecated
-  protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context, String table) {
-    return InputConfigurator.getFetchedColumns(CLASS,getConfiguration(context),table);
+  protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
+    return InputConfigurator.getFetchedColumns(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * Encode an iterator on the default all tables for this job.
    * 
@@ -423,42 +386,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   @Deprecated
   public static void addIterator(Job job, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS,job.getConfiguration(),cfg);
-  }
-
-  /**
-   * Encode an iterator on the input for this job for the specified table.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param table
-   *          the table for which to add the iterator
-   * @param cfg
-   *          the configuration of the iterator
-   * @since 1.6.0
-   */
-  @Deprecated
-  public static void addIterator(Job job, String table, IteratorSetting cfg) {
-    InputConfigurator.addIterator(CLASS, job.getConfiguration(), table, cfg);
+    InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
   }
-
-  /**
-   * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration for the specific table. Any default iterators will be
-   * included in the return.
-   * 
-   * @param context
-   *          the Hadoop context for the configured job
-   * @param table
-   *          the table for which to return the iterators
-   * @return a list of iterators for the given table
-   * @since 1.6.0
-   * @see #addIterator(Job, String, IteratorSetting)
-   */
-  @Deprecated
-  protected static List<IteratorSetting> getIterators(JobContext context, String table) {
-    return InputConfigurator.getIterators(CLASS, getConfiguration(context), table);
-  }
-
+  
   /**
    * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration. This will only return iterators that have not been set
    * for a specific table.
@@ -471,9 +401,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    */
   @Deprecated
   protected static List<IteratorSetting> getIterators(JobContext context) {
-    return InputConfigurator.getDefaultIterators(CLASS,getConfiguration(context));
+    return InputConfigurator.getIterators(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * 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. *
@@ -492,7 +422,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
     InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration has auto-adjust ranges enabled.
    * 
@@ -506,21 +436,20 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean getAutoAdjustRanges(JobContext context) {
     return InputConfigurator.getAutoAdjustRanges(CLASS, getConfiguration(context));
   }
-
-
+  
   protected static void setTableQueryConfigurations(JobContext job, TableQueryConfig... configs) {
     checkNotNull(configs);
-    InputConfigurator.setTableQueryConfiguration(CLASS, getConfiguration(job), configs);
+    InputConfigurator.setTableQueryConfigs(CLASS, getConfiguration(job), configs);
   }
-
+  
   public static List<TableQueryConfig> getTableQueryConfigurations(JobContext job) {
-    return InputConfigurator.getTableQueryConfigurations(CLASS, getConfiguration(job));
+    return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job));
   }
-
+  
   protected static TableQueryConfig getTableQueryConfiguration(JobContext job, String tableName) {
-    return InputConfigurator.getTableQueryConfiguration(CLASS, getConfiguration(job), tableName);
+    return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job), tableName);
   }
-
+  
   /**
    * Controls the use of the {@link IsolatedScanner} in this job.
    * 
@@ -536,7 +465,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setScanIsolation(Job job, boolean enableFeature) {
     InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration has isolation enabled.
    * 
@@ -549,7 +478,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean isIsolated(JobContext context) {
     return InputConfigurator.isIsolated(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * 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.
@@ -566,7 +495,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setLocalIterators(Job job, boolean enableFeature) {
     InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration uses local iterators.
    * 
@@ -579,7 +508,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean usesLocalIterators(JobContext context) {
     return InputConfigurator.usesLocalIterators(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * <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
@@ -614,7 +543,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   public static void setOfflineTableScan(Job job, boolean enableFeature) {
     InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration has the offline table scan feature enabled.
    * 
@@ -627,7 +556,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static boolean isOfflineScan(JobContext context) {
     return InputConfigurator.isOfflineScan(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * Initializes an Accumulo {@link TabletLocator} based on the configuration.
    * 
@@ -643,7 +572,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
     return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context), table);
   }
-
+  
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
   /**
    * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
@@ -657,7 +586,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   protected static void validateOptions(JobContext context) throws IOException {
     InputConfigurator.validateOptions(CLASS, getConfiguration(context));
   }
-
+  
   /**
    * An abstract base class to be used to create {@link RecordReader} instances that convert from Accumulo {@link Key}/{@link Value} pairs to the user's K/V
    * types.
@@ -674,62 +603,63 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     protected long numKeysRead;
     protected Iterator<Entry<Key,Value>> scannerIterator;
     protected RangeInputSplit split;
-
+    
     /**
-     * Apply the configured iterators from the configuration to the scanner. This applies both the
-     * default iterators and the per-table iterators.
+     * Apply the configured iterators from the configuration to the scanner. This applies both the default iterators and the per-table iterators.
      * 
      * @param context
      *          the Hadoop context for the configured job
      * @param scanner
      *          the scanner to configure
-     *  @param tableName
+     * @param tableName
      *          the table name for which to set up the iterators
      */
     protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
-      List<IteratorSetting> iterators = getIterators(context, tableName); // default iterators will be included
+      TableQueryConfig config = getTableQueryConfiguration(context, tableName);
+      List<IteratorSetting> iterators = config.getIterators();
       for (IteratorSetting iterator : iterators)
         scanner.addScanIterator(iterator);
     }
-
-
+    
     /**
      * Initialize a scanner over the given input split using this task attempt configuration.
      */
     @Override
     public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
-
+      
       Scanner scanner;
       split = (RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + split.getRange());
       Instance instance = getInstance(attempt);
       String principal = getPrincipal(attempt);
-
+      
       TableQueryConfig tableConfig = getTableQueryConfiguration(attempt, split.getTableName());
-
+      
       // in case the table name changed, we can still use the previous name for terms of configuration,
       // but for the scanner, we'll need to reference the new table name.
-      String actualNameForId = null;
-      try{
-        actualNameForId = Tables.getTableName(instance, split.getTableId());
-        if(!actualNameForId.equals(split.getTableName()))
-          log.debug("Table name changed from " + split.getTableName() + " to " + actualNameForId);
-      }catch(TableNotFoundException e){
-        throw new IOException("The specified table was not found for id=" + split.getTableId());
+      String actualNameForId = split.getTableName();
+      if(!(instance instanceof MockInstance)) {   // Really, the Tables helper class should not be tied to Zookeeper
+        try {
+          actualNameForId = Tables.getTableName(instance, split.getTableId());
+          if (!actualNameForId.equals(split.getTableName()))
+            log.debug("Table name changed from " + split.getTableName() + " to " + actualNameForId);
+        } catch (TableNotFoundException e) {
+          throw new IOException("The specified table was not found for id=" + split.getTableId());
+        }
       }
 
       AuthenticationToken token = getAuthenticationToken(attempt);
       Authorizations authorizations = getScanAuthorizations(attempt);
       try {
         log.debug("Creating connector with user: " + principal);
-
+        
         Connector conn = instance.getConnector(principal, token);
-        log.debug("Creating scanner for table: " + actualNameForId);
+        log.debug("Creating scanner for table: " + split.getTableName());
         log.debug("Authorizations are: " + authorizations);
         if (isOfflineScan(attempt)) {
           scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
         } else {
-          scanner = conn.createScanner(actualNameForId, authorizations);
+          scanner = conn.createScanner(split.getTableName(), authorizations);
         }
         if (isIsolated(attempt)) {
           log.info("Creating isolated scanner");
@@ -743,7 +673,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       } catch (Exception e) {
         throw new IOException(e);
       }
-
+      
       // setup a scanner within the bounds of this split
       for (Pair<Text,Text> c : tableConfig.getColumns()) {
         if (c.getSecond() != null) {
@@ -754,64 +684,64 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           scanner.fetchColumnFamily(c.getFirst());
         }
       }
-
+      
       scanner.setRange(split.getRange());
       numKeysRead = 0;
-
+      
       // do this last after setting all scanner options
       scannerIterator = scanner.iterator();
     }
-
+    
     @Override
     public void close() {}
-
+    
     @Override
     public float getProgress() throws IOException {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return split.getProgress(currentKey);
     }
-
+    
     protected K currentK = null;
     protected V currentV = null;
     protected Key currentKey = null;
     protected Value currentValue = null;
-
+    
     @Override
     public K getCurrentKey() throws IOException, InterruptedException {
       return currentK;
     }
-
+    
     @Override
     public V getCurrentValue() throws IOException, InterruptedException {
       return currentV;
     }
   }
-
+  
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, String tableName, List<Range> ranges) throws TableNotFoundException,
       AccumuloException, AccumuloSecurityException {
-
+    
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-
+    
     Instance instance = getInstance(context);
     Connector conn = instance.getConnector(getPrincipal(context), getAuthenticationToken(context));
     String tableId = Tables.getTableId(instance, tableName);
-
+    
     if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
       Tables.clearCache(instance);
       if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
         throw new AccumuloException("Table is online " + tableName + "(" + 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);
       TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
@@ -819,7 +749,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
       scanner.fetchColumnFamily(TabletsSection.FutureLocationColumnFamily.NAME);
       scanner.setRange(metadataRange);
-
+      
       RowIterator rowIter = new RowIterator(scanner);
       KeyExtent lastExtent = null;
       while (rowIter.hasNext()) {
@@ -827,89 +757,88 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         String last = "";
         KeyExtent extent = null;
         String location = null;
-
+        
         while (row.hasNext()) {
           Entry<Key,Value> entry = row.next();
           Key key = entry.getKey();
-
+          
           if (key.getColumnFamily().equals(TabletsSection.LastLocationColumnFamily.NAME)) {
             last = entry.getValue().toString();
           }
-
+          
           if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME)
               || key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME)) {
             location = entry.getValue().toString();
           }
-
+          
           if (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;
   }
-
+  
   /**
    * Gets the splits of the tables that have been set on the job.
-   *
+   * 
    * @param conf
    *          the configuration of the job
-   * @return  the splits from the tables based on the ranges.
+   * @return the splits from the tables based on the ranges.
    * @throws IOException
-   *          if a table set on the job doesn't exist or an error occurs
-   *          initializing the tablet locator
+   *           if a table set on the job doesn't exist or an error occurs initializing the tablet locator
    */
   public List<InputSplit> getSplits(JobContext conf) throws IOException {
     log.setLevel(getLogLevel(conf));
     validateOptions(conf);
-
+    
     LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
     List<TableQueryConfig> tableConfigs = getTableQueryConfigurations(conf);
     for (TableQueryConfig tableConfig : tableConfigs) {
-
-      boolean autoAdjust = getAutoAdjustRanges(conf); // TODO: Put this in the table config object
+      
+      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
       String tableId = null;
       List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
       if (ranges.isEmpty()) {
         ranges = new ArrayList<Range>(1);
         ranges.add(new Range());
       }
-
+      
       // get the metadata information for these ranges
       Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
       TabletLocator tl;
@@ -920,7 +849,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
             // Some tablets were still online, try again
             UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
             binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
-
+            
           }
         } else {
           Instance instance = getInstance(conf);
@@ -928,7 +857,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
           tl.invalidateCache();
           Credentials creds = new Credentials(getPrincipal(conf), getAuthenticationToken(conf));
-
+          
           while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
             if (!(instance instanceof MockInstance)) {
               if (!Tables.exists(instance, tableId))
@@ -946,12 +875,12 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       } catch (Exception e) {
         throw new IOException(e);
       }
-
+      
       HashMap<Range,ArrayList<String>> splitsToAdd = null;
-
+      
       if (!autoAdjust)
         splitsToAdd = new HashMap<Range,ArrayList<String>>();
-
+      
       HashMap<String,String> hostNameCache = new HashMap<String,String>();
       for (Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
         String ip = tserverBin.getKey().split(":", 2)[0];
@@ -978,14 +907,14 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
           }
         }
       }
-
+      
       if (!autoAdjust)
         for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
           splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
     }
     return splits;
   }
-
+  
   /**
    * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
    */
@@ -994,51 +923,51 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
     private String[] locations;
     private String tableId;
     private String tableName;
-
+    
     public RangeInputSplit() {
       range = new Range();
       locations = new String[0];
       tableId = "";
       tableName = "";
     }
-
+    
     public RangeInputSplit(RangeInputSplit split) throws IOException {
       this.setRange(split.getRange());
       this.setLocations(split.getLocations());
       this.setTableName(split.getTableName());
     }
-
+    
     protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
       this.range = range;
       this.locations = locations;
       this.tableName = table;
       this.tableId = tableId;
     }
-
+    
     public Range getRange() {
       return range;
     }
-
+    
     public void setRange(Range range) {
       this.range = range;
     }
-
+    
     public String getTableName() {
       return tableName;
     }
-
+    
     public void setTableName(String tableName) {
       this.tableName = tableName;
     }
-
+    
     public void setTableId(String tableId) {
       this.tableId = tableId;
     }
-
+    
     public String getTableId() {
       return tableId;
     }
-
+    
     private static byte[] extractBytes(ByteSequence seq, int numBytes) {
       byte[] bytes = new byte[numBytes + 1];
       bytes[0] = 0;
@@ -1050,7 +979,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       }
       return bytes;
     }
-
+    
     public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
       int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
       BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
@@ -1058,7 +987,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
       return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
     }
-
+    
     public float getProgress(Key currentKey) {
       if (currentKey == null)
         return 0f;
@@ -1077,7 +1006,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       // if we can't figure it out, then claim no progress
       return 0f;
     }
-
+    
     /**
      * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
      */
@@ -1087,29 +1016,29 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
       int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
       long diff = 0;
-
+      
       byte[] start = startRow.getBytes();
       byte[] stop = stopRow.getBytes();
       for (int i = 0; i < maxCommon; ++i) {
         diff |= 0xff & (start[i] ^ stop[i]);
         diff <<= Byte.SIZE;
       }
-
+      
       if (startRow.getLength() != stopRow.getLength())
         diff |= 0xff;
-
+      
       return diff + 1;
     }
-
+    
     @Override
     public String[] getLocations() throws IOException {
       return locations;
     }
-
+    
     public void setLocations(String[] locations) {
       this.locations = locations;
     }
-
+    
     @Override
     public void readFields(DataInput in) throws IOException {
       range.readFields(in);
@@ -1119,7 +1048,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       for (int i = 0; i < numLocs; ++i)
         locations[i] = in.readUTF();
     }
-
+    
     @Override
     public void write(DataOutput out) throws IOException {
       range.write(out);
@@ -1129,7 +1058,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         out.writeUTF(locations[i]);
     }
   }
-
+  
   // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
   static Configuration getConfiguration(JobContext context) {
     try {
@@ -1141,5 +1070,4 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       throw new RuntimeException(e);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e6a7c962/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
index 29795af..5e02177 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
@@ -25,8 +25,10 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
-import java.util.TreeMap;
+import java.util.Set;
+import java.util.StringTokenizer;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -65,7 +67,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * @since 1.5.0
    */
   public static enum ScanOpts {
-    TABLE, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS
+    TABLE, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS, TABLE_CONFIGS
   }
 
   /**
@@ -201,6 +203,42 @@ public class InputConfigurator extends ConfiguratorBase {
     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.5.0
+   * @see #addIterator(Class, Configuration, IteratorSetting)
+   */
+  public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
+    String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
+
+    // If no iterators are present, return an empty list
+    if (iterators == null || iterators.isEmpty())
+      return new ArrayList<IteratorSetting>();
+
+    // Compose the set of iterators encoded in the job configuration
+    StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR);
+    List<IteratorSetting> list = new ArrayList<IteratorSetting>();
+    try {
+      while (tokens.hasMoreTokens()) {
+        String itstring = tokens.nextToken();
+        ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(itstring.getBytes()));
+        list.add(new IteratorSetting(new DataInputStream(bais)));
+        bais.close();
+      }
+    } catch (IOException e) {
+      throw new IllegalArgumentException("couldn't decode iterator settings");
+    }
+    return list;
+  }
+
+
   /**
    * Restricts the columns that will be mapped over for this job. This applies the columns to all tables that have been set on the job.
    * 
@@ -233,6 +271,29 @@ public class InputConfigurator extends ConfiguratorBase {
     conf.setStrings(enumToConfKey(implementingClass, ScanOpts.COLUMNS), 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.5.0
+   * @see #fetchColumns(Class, Configuration, Collection)
+   */
+  public static Set<Pair<Text,Text>> getFetchedColumns(Class<?> implementingClass, Configuration conf) {
+    Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
+    for (String col : conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.COLUMNS))) {
+      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()));
+      columns.add(new Pair<Text,Text>(cf, cq));
+    }
+    return columns;
+  }
+
   /**
    * Encode an iterator on the input for all tables associated with this job.
    * 
@@ -269,7 +330,7 @@ public class InputConfigurator extends ConfiguratorBase {
       iterators = iterators.concat(StringUtils.COMMA_STR + newIter);
     }
     // Store the iterators w/ the job
-    conf.set(confKey, iterators);
+    conf.set(confKey,iterators);
   }
 
   /**
@@ -438,7 +499,7 @@ public class InputConfigurator extends ConfiguratorBase {
     return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
   }
 
-  public static void setTableQueryConfiguration(Class<?> implementingClass, Configuration conf, TableQueryConfig... tconf) {
+  public static void setTableQueryConfigs(Class<?> implementingClass, Configuration conf, TableQueryConfig... tconf) {
     List<String> tableQueryConfigStrings = new ArrayList<String>();
     for(TableQueryConfig queryConfig : tconf) {
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -449,14 +510,14 @@ public class InputConfigurator extends ConfiguratorBase {
       }
       tableQueryConfigStrings.add(new String(Base64.encodeBase64(baos.toByteArray())));
     }
-    String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE);
+    String confKey = enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS);
     conf.setStrings(confKey, tableQueryConfigStrings.toArray(new String[0]));
   }
 
-  public static List<TableQueryConfig> getTableQueryConfigurations(Class<?> implementingClass, Configuration conf) {
+  public static List<TableQueryConfig> getTableQueryConfigs(Class<?> implementingClass, Configuration conf) {
     List<TableQueryConfig> configs = new ArrayList<TableQueryConfig>();
-    Collection<String> configStrings = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.TABLE));
-    if(configStrings == null) {
+    Collection<String> configStrings = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.TABLE_CONFIGS));
+    if(configStrings != null) {
       for(String str : configStrings) {
         try{
           byte[] bytes = Base64.decodeBase64(str.getBytes());
@@ -468,24 +529,23 @@ public class InputConfigurator extends ConfiguratorBase {
         }
       }
     }
+    TableQueryConfig defaultQueryConfig;
+    try {
+      defaultQueryConfig = getDefaultTableConfig(implementingClass, conf);
+    } catch(IOException e) {
+      throw new IllegalStateException("There was an error deserializing the default table configuration.");
+    }
+    if(defaultQueryConfig != null)
+      configs.add(defaultQueryConfig);
+
     return configs;
   }
 
-  public static TableQueryConfig getTableQueryConfiguration(Class<?> implementingClass, Configuration conf, String tableName) {
-    Collection<String> configStrings = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.TABLE));
-    if(configStrings == null) {
-      for(String str : configStrings) {
-        if(str.equals(tableName)) {
-          try{
-            byte[] bytes = Base64.decodeBase64(str.getBytes());
-            ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-            TableQueryConfig config = new TableQueryConfig(new DataInputStream(bais));
-            bais.close();
-            return config;
-          } catch(IOException e) {
-            throw new IllegalStateException("The table query configurations could not be deserialized from the given configuration");
-          }
-        }
+  public static TableQueryConfig getTableQueryConfigs(Class<?> implementingClass, Configuration conf, String tableName) {
+    List<TableQueryConfig> queryConfigs = getTableQueryConfigs(implementingClass,conf);
+    for(TableQueryConfig queryConfig : queryConfigs) {
+      if(queryConfig.getTableName().equals(tableName)) {
+        return queryConfig;
       }
     }
     return null;
@@ -509,7 +569,7 @@ public class InputConfigurator extends ConfiguratorBase {
     String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
     if ("MockInstance".equals(instanceType))
       return new MockTabletLocator();
-    Instance instance = getInstance(implementingClass, conf);
+    Instance instance = getInstance(implementingClass,conf);
     return TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
   }
 
@@ -539,19 +599,19 @@ public class InputConfigurator extends ConfiguratorBase {
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
 
-      for (TableQueryConfig tableConfig : getTableQueryConfigurations(implementingClass, conf)) {
+      for (TableQueryConfig tableConfig : getTableQueryConfigs(implementingClass,conf)) {
         if (!c.securityOperations().hasTablePermission(getPrincipal(implementingClass, conf), tableConfig.getTableName(), TablePermission.READ))
           throw new IOException("Unable to access table");
       }
 
-      for (TableQueryConfig tableConfig : getTableQueryConfigurations(implementingClass,conf)) {
+      for (TableQueryConfig tableConfig : getTableQueryConfigs(implementingClass,conf)) {
         if(!tableConfig.shouldUseLocalIterators()) {
-//        if (!conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false)) {
-          // validate that any scan-time iterators can be loaded by the the tablet servers
-          for (IteratorSetting iter : tableConfig.getIterators()) { // TODO: These iterators need to be separated by table
-            if (!c.tableOperations().testClassLoad(tableConfig.getTableName(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
-              throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
+          if(tableConfig.getIterators() != null) {
+            for (IteratorSetting iter : tableConfig.getIterators()) {
+              if (!c.tableOperations().testClassLoad(tableConfig.getTableName(), iter.getIteratorClass(), SortedKeyValueIterator.class.getName()))
+                throw new AccumuloException("Servers are unable to load " + iter.getIteratorClass() + " as a " + SortedKeyValueIterator.class.getName());
 
+            }
           }
         }
       }
@@ -567,4 +627,29 @@ public class InputConfigurator extends ConfiguratorBase {
     }
   }
 
+
+  protected static TableQueryConfig getDefaultTableConfig(Class<?> implementingClass, Configuration conf) throws IOException {
+    String tableName = getInputTableName(implementingClass, conf);
+    if(tableName != null) {
+      TableQueryConfig queryConfig = new TableQueryConfig(getInputTableName(implementingClass, conf));
+      List<IteratorSetting> itrs = getIterators(implementingClass, conf);
+      if(itrs != null)
+        queryConfig.setIterators(itrs);
+      Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
+      if(columns != null)
+        queryConfig.setColumns(columns);
+      List<Range> ranges = getRanges(implementingClass, conf);
+      if(ranges != null)
+        queryConfig.setRanges(ranges);
+
+      queryConfig.setAutoAdjustRanges(getAutoAdjustRanges(implementingClass, conf))
+              .setUseIsolatedScanners(isIsolated(implementingClass, conf))
+              .setUseLocalIterators(usesLocalIterators(implementingClass, conf))
+              .setOfflineScan(isOfflineScan(implementingClass, conf));
+      return queryConfig;
+    }
+    return null;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e6a7c962/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java b/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
index 00f4dc0..d6476da 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/TableQueryConfig.java
@@ -27,6 +27,7 @@ public class TableQueryConfig implements Writable {
   private boolean autoAdjustRanges = true;
   private boolean useLocalIterators = false;
   private boolean useIsolatedScanners = false;
+  private boolean offlineScan = false;
   
   public TableQueryConfig(String tableName) {
     checkNotNull(tableName);
@@ -67,20 +68,29 @@ public class TableQueryConfig implements Writable {
     return this;
   }
 
+  public boolean isOfflineScan(){
+    return offlineScan;
+  }
+
+  public TableQueryConfig setOfflineScan(boolean offlineScan){
+    this.offlineScan=offlineScan;
+    return this;
+  }
+
   public String getTableName(){
     return tableName;
   }
 
   public List<IteratorSetting> getIterators(){
-    return iterators;
+    return iterators != null ? iterators : new ArrayList<IteratorSetting>();
   }
 
   public List<Range> getRanges(){
-    return ranges;
+    return ranges != null ? ranges : new ArrayList<Range>();
   }
 
   public Set<Pair<Text,Text>> getColumns(){
-    return columns;
+    return columns != null ? columns : new HashSet<Pair<Text,Text>>();
   }
 
   public boolean shouldAutoAdjustRanges(){
@@ -179,6 +189,9 @@ public class TableQueryConfig implements Writable {
     TableQueryConfig that=(TableQueryConfig)o;
 
     if(autoAdjustRanges!=that.autoAdjustRanges) return false;
+    if(offlineScan!=that.offlineScan) return false;
+    if(useIsolatedScanners!=that.useIsolatedScanners) return false;
+    if(useLocalIterators!=that.useLocalIterators) return false;
     if(columns!=null?!columns.equals(that.columns):that.columns!=null) return false;
     if(iterators!=null?!iterators.equals(that.iterators):that.iterators!=null) return false;
     if(ranges!=null?!ranges.equals(that.ranges):that.ranges!=null) return false;
@@ -194,6 +207,9 @@ public class TableQueryConfig implements Writable {
     result=31*result+(ranges!=null?ranges.hashCode():0);
     result=31*result+(columns!=null?columns.hashCode():0);
     result=31*result+(autoAdjustRanges?1:0);
+    result=31*result+(useLocalIterators?1:0);
+    result=31*result+(useIsolatedScanners?1:0);
+    result=31*result+(offlineScan?1:0);
     return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e6a7c962/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index 0f6655a..96a67a6 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -311,127 +311,25 @@ public class AccumuloInputFormatTest {
     assertNull(e2);
   }
 
-  /**
-   * Asserts that the configuration contains the expected ranges for the tables.
-   */
   @Test
-  public void testMultitableRangeSerialization() throws Throwable {
-    List<String> tables = Arrays.asList("t1", "t2", "t3");
-    Job job = new Job(new Configuration());
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    job.setMapperClass(MRTester.TestMapper.class);
-    job.setNumReduceTasks(0);
-    AccumuloInputFormat.setConnectorInfo(job, "root", new PasswordToken(new byte[0]));
-    AccumuloInputFormat.setScanAuthorizations(job, new Authorizations());
-    AccumuloInputFormat.setMockInstance(job, "testmapinstance");
-
-    HashMap<String,Collection<Range>> tblRanges = new HashMap<String,Collection<Range>>();
-    for (String tbl : tables) {
-      List<Range> ranges = Arrays.asList(new Range("a", "b"), new Range("c", "d"), new Range("e", "f"));
-      tblRanges.put(tbl, ranges);
-    }
+  public void testTableQueryConfigSerialization() throws IOException{
 
-    Range defaultRange = new Range("0", "1");
-
-    try {
-      AccumuloInputFormat.setRanges(job, tblRanges);
-      fail("Exception should have been thrown.");
-    } catch(IllegalStateException e) {}
+    Job job = new Job();
 
-    AccumuloInputFormat.setInputTableNames(job, tables);
+    TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1)
+            .setRanges(Collections.singletonList(new Range("a", "b")))
+            .setColumns(Collections.singleton(new Pair<Text,Text>(new Text("CF1"), new Text("CQ1"))))
+            .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 
-    // set a default range
-    AccumuloInputFormat.setRanges(job, Collections.singleton(defaultRange));
-    AccumuloInputFormat.setRanges(job, tblRanges);
-    Map<String,List<Range>> configuredRanges = AccumuloInputFormat.getRanges(job);
+    TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2)
+            .setRanges(Collections.singletonList(new Range("a", "b")))
+            .setColumns(Collections.singleton(new Pair<Text,Text>(new Text("CF1"), new Text("CQ1"))))
+            .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 
-    for (Map.Entry<String,List<Range>> cfgRange : configuredRanges.entrySet()) {
-      String tbl = cfgRange.getKey();
-      HashSet<Range> originalRanges = new HashSet<Range>(tblRanges.remove(tbl));
-      originalRanges.add(defaultRange);
-      HashSet<Range> retrievedRanges = new HashSet<Range>(cfgRange.getValue());
-      assertEquals(originalRanges.size(), retrievedRanges.size());
-      assertTrue(originalRanges.containsAll(retrievedRanges));
-      assertTrue(retrievedRanges.containsAll(originalRanges));
-    }
-  }
+    AccumuloInputFormat.setTableQueryConfigurations(job, table1, table2);
 
-  /**
-   * Asserts that the configuration contains the expected iterators for the tables.
-   */
-  @Test
-  public void testMultitableIteratorSerialization() throws Throwable {
-    HashSet<String> tables = new HashSet<String>(Arrays.asList("t1", "t2"));
-    Job job = new Job(new Configuration());
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    job.setMapperClass(MRTester.TestMapper.class);
-    job.setNumReduceTasks(0);
-    AccumuloInputFormat.setConnectorInfo(job, "root", new PasswordToken(new byte[0]));
-    AccumuloInputFormat.setScanAuthorizations(job, new Authorizations());
-
-    // create + set iterators on configuration and build expected reference set
-    IteratorSetting isetting1 = new IteratorSetting(1, "name1", "class1");
-    IteratorSetting isetting2 = new IteratorSetting(2, "name2", "class2");
-    IteratorSetting isetting3 = new IteratorSetting(2, "name3", "class3");
-
-    try {
-      AccumuloInputFormat.addIterator(job, "t1", isetting1);
-      fail("Exception should have been thrown.");
-    } catch(IllegalStateException e) {}
-
-    AccumuloInputFormat.setInputTableNames(job, tables);
-
-    AccumuloInputFormat.addIterator(job, "t1", isetting1);
-    AccumuloInputFormat.addIterator(job, "t2", isetting2);
-    AccumuloInputFormat.addIterator(job, isetting3);
-
-    // verify per-table iterators
-    List<IteratorSetting> t1iters = AccumuloInputFormat.getIterators(job, "t1");
-    List<IteratorSetting> t2iters = AccumuloInputFormat.getIterators(job, "t2");
-    assertFalse(t1iters.isEmpty());
-    assertEquals(isetting1, t1iters.get(1));
-    assertEquals(isetting3, t1iters.get(0));
-    assertEquals(isetting2, t2iters.get(1));
-    assertEquals(isetting3, t2iters.get(0));
+    assertEquals(table1, AccumuloInputFormat.getTableQueryConfiguration(job, TEST_TABLE_1));
+    assertEquals(table2, AccumuloInputFormat.getTableQueryConfiguration(job, TEST_TABLE_2));
   }
 
-  @Test
-  public void testMultitableColumnSerialization() throws IOException, AccumuloSecurityException {
-    HashSet<String> tables = new HashSet<String>(Arrays.asList("t1", "t2"));
-    Job job = new Job(new Configuration());
-    job.setInputFormatClass(AccumuloInputFormat.class);
-    job.setMapperClass(MRTester.TestMapper.class);
-    job.setNumReduceTasks(0);
-    AccumuloInputFormat.setConnectorInfo(job, "root", new PasswordToken(new byte[0]));
-
-    AccumuloInputFormat.setScanAuthorizations(job, new Authorizations());
-
-    Map<String,Collection<Pair<Text,Text>>> columns = new HashMap<String,Collection<Pair<Text,Text>>>();
-    HashSet<Pair<Text,Text>> t1cols = new HashSet<Pair<Text,Text>>();
-    t1cols.add(new Pair(new Text("a"), new Text("b")));
-    HashSet<Pair<Text,Text>> t2cols = new HashSet<Pair<Text,Text>>();
-    t2cols.add(new Pair(new Text("b"), new Text("c")));
-    columns.put("t1", t1cols);
-    columns.put("t2", t2cols);
-
-    Pair<Text,Text> defaultColumn = new Pair(new Text("c"), new Text("d"));
-
-    try {
-      AccumuloInputFormat.fetchColumns(job, columns);
-      fail("Exception should have been thrown.");
-    } catch(IllegalStateException e) {}
-
-    AccumuloInputFormat.setInputTableNames(job, tables);
-
-    AccumuloInputFormat.fetchColumns(job, Collections.singleton(defaultColumn));
-    AccumuloInputFormat.fetchColumns(job, columns);
-
-    columns.get("t1").add(defaultColumn);
-    columns.get("t2").add(defaultColumn);
-
-    Collection<Pair<Text,Text>> t1actual = AccumuloInputFormat.getFetchedColumns(job, "t1");
-    assertEquals(columns.get("t1"), t1actual);
-    Collection<Pair<Text,Text>> t2actual = AccumuloInputFormat.getFetchedColumns(job, "t2");
-    assertEquals(columns.get("t2"), t2actual);
-  }
 }