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

[11/11] git commit: Adding the following: - Deprecation to InputConfigurator, mapred.InputFormatBase, mapreduce.InputFormatBase - Comments to TableQueryConfig - Multi-table support to mapred.InputFormatBase

Adding the following:
- Deprecation to InputConfigurator, mapred.InputFormatBase, mapreduce.InputFormatBase
- Comments to TableQueryConfig
- Multi-table support to mapred.InputFormatBase

ACCUMULO-391


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

Branch: refs/heads/ACCUMULO-391
Commit: 3227a822379718d6c1297f11d7af37a716f78a60
Parents: 6648e8a
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Tue Oct 1 23:20:34 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 1 23:20:34 2013 -0400

----------------------------------------------------------------------
 .../core/client/mapred/AccumuloInputFormat.java |   1 +
 .../core/client/mapred/InputFormatBase.java     | 414 +++++++++++--------
 .../client/mapreduce/AccumuloInputFormat.java   |   2 +-
 .../core/client/mapreduce/InputFormatBase.java  |  52 ++-
 .../mapreduce/lib/util/InputConfigurator.java   |  44 +-
 .../accumulo/core/conf/TableQueryConfig.java    | 185 +++++++--
 .../client/mapred/AccumuloInputFormatTest.java  |  85 +++-
 .../mapreduce/AccumuloInputFormatTest.java      |  13 +-
 .../core/conf/TableQueryConfigTest.java         |   4 +-
 9 files changed, 575 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
index bbbd0c3..b3b9fc7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.mapred.Reporter;
  * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
  * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, String, String)} OR {@link AccumuloInputFormat#setMockInstance(JobConf, String)}
+ * <li>{@link AccumuloInputFormat#setTableQueryConfigs(JobConf, org.apache.accumulo.core.conf.TableQueryConfig...)}</li>
  * </ul>
  * 
  * Other static methods are optional.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/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 c5831a1..ae51581 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
@@ -48,6 +48,7 @@ import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 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.conf.TableQueryConfig;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.PartialKey;
@@ -69,6 +70,8 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * This abstract {@link InputFormat} class allows MapReduce jobs to use Accumulo as the source of K,V pairs.
  * <p>
@@ -80,10 +83,10 @@ import org.apache.log4j.Logger;
  * See {@link AccumuloInputFormat} for an example implementation.
  */
 public abstract class InputFormatBase<K,V> implements 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.
    * 
@@ -103,7 +106,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public static void setConnectorInfo(JobConf job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
     InputConfigurator.setConnectorInfo(CLASS, job, principal, token);
   }
-
+  
   /**
    * Sets the connector information needed to communicate with Accumulo in this job.
    * 
@@ -122,7 +125,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public static void setConnectorInfo(JobConf job, String principal, String tokenFile) throws AccumuloSecurityException {
     InputConfigurator.setConnectorInfo(CLASS, job, principal, tokenFile);
   }
-
+  
   /**
    * Determines if the connector has been configured.
    * 
@@ -135,7 +138,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static Boolean isConnectorInfoSet(JobConf job) {
     return InputConfigurator.isConnectorInfoSet(CLASS, job);
   }
-
+  
   /**
    * Gets the user name from the configuration.
    * 
@@ -148,7 +151,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static String getPrincipal(JobConf job) {
     return InputConfigurator.getPrincipal(CLASS, job);
   }
-
+  
   /**
    * Gets the serialized token class from either the configuration or the token file.
    * 
@@ -159,7 +162,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static String getTokenClass(JobConf job) {
     return getAuthenticationToken(job).getClass().getName();
   }
-
+  
   /**
    * Gets the serialized token from either the configuration or the token file.
    * 
@@ -170,7 +173,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static byte[] getToken(JobConf job) {
     return AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
   }
-
+  
   /**
    * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
    * 
@@ -184,7 +187,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static AuthenticationToken getAuthenticationToken(JobConf job) {
     return InputConfigurator.getAuthenticationToken(CLASS, job);
   }
-
+  
   /**
    * Configures a {@link ZooKeeperInstance} for this job.
    * 
@@ -199,7 +202,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
     InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
   }
-
+  
   /**
    * Configures a {@link MockInstance} for this job.
    * 
@@ -212,7 +215,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public static void setMockInstance(JobConf job, String instanceName) {
     InputConfigurator.setMockInstance(CLASS, job, instanceName);
   }
-
+  
   /**
    * Initializes an Accumulo {@link Instance} based on the configuration.
    * 
@@ -226,7 +229,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static Instance getInstance(JobConf job) {
     return InputConfigurator.getInstance(CLASS, job);
   }
-
+  
   /**
    * Sets the log level for this job.
    * 
@@ -239,7 +242,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public static void setLogLevel(JobConf job, Level level) {
     InputConfigurator.setLogLevel(CLASS, job, level);
   }
-
+  
   /**
    * Gets the log level from this configuration.
    * 
@@ -252,7 +255,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static Level getLogLevel(JobConf job) {
     return InputConfigurator.getLogLevel(CLASS, job);
   }
-
+  
   /**
    * Sets the name of the input table, over which this job will scan.
    * 
@@ -261,11 +264,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @param tableName
    *          the table to use when the tablename is null in the write call
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setInputTableName(JobConf job, String tableName) {
     InputConfigurator.setInputTableName(CLASS, job, tableName);
   }
-
+  
   /**
    * Gets the table name from the configuration.
    * 
@@ -274,11 +279,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return the table name
    * @since 1.5.0
    * @see #setInputTableName(JobConf, String)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static String getInputTableName(JobConf job) {
     return InputConfigurator.getInputTableName(CLASS, job);
   }
-
+  
   /**
    * Sets the {@link Authorizations} used to scan. Must be a subset of the user's authorization. Defaults to the empty set.
    * 
@@ -291,7 +298,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public static void setScanAuthorizations(JobConf job, Authorizations auths) {
     InputConfigurator.setScanAuthorizations(CLASS, job, auths);
   }
-
+  
   /**
    * Gets the authorizations to set for the scans from the configuration.
    * 
@@ -304,7 +311,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static Authorizations getScanAuthorizations(JobConf job) {
     return InputConfigurator.getScanAuthorizations(CLASS, job);
   }
-
+  
   /**
    * Sets the input ranges to scan for this job. If not set, the entire table will be scanned.
    * 
@@ -313,11 +320,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @param ranges
    *          the ranges that will be mapped over
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setRanges(JobConf job, Collection<Range> ranges) {
     InputConfigurator.setRanges(CLASS, job, ranges);
   }
-
+  
   /**
    * Gets the ranges to scan over from a job.
    * 
@@ -328,11 +337,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    *           if the ranges have been encoded improperly
    * @since 1.5.0
    * @see #setRanges(JobConf, Collection)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static List<Range> getRanges(JobConf job) throws IOException {
     return InputConfigurator.getRanges(CLASS, job);
   }
-
+  
   /**
    * Restricts the columns that will be mapped over for this job.
    * 
@@ -342,11 +353,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    *          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.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void fetchColumns(JobConf job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     InputConfigurator.fetchColumns(CLASS, job, columnFamilyColumnQualifierPairs);
   }
-
+  
   /**
    * Gets the columns to be mapped over from this job.
    * 
@@ -355,11 +368,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return a set of columns
    * @since 1.5.0
    * @see #fetchColumns(JobConf, Collection)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobConf job) {
     return InputConfigurator.getFetchedColumns(CLASS, job);
   }
-
+  
   /**
    * Encode an iterator on the input for this job.
    * 
@@ -368,11 +383,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @param cfg
    *          the configuration of the iterator
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void addIterator(JobConf job, IteratorSetting cfg) {
     InputConfigurator.addIterator(CLASS, job, cfg);
   }
-
+  
   /**
    * Gets a list of the iterator settings (for iterators to apply to a scanner) from this configuration.
    * 
@@ -381,11 +398,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return a list of iterators
    * @since 1.5.0
    * @see #addIterator(JobConf, IteratorSetting)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static List<IteratorSetting> getIterators(JobConf job) {
     return InputConfigurator.getIterators(CLASS, job);
   }
-
+  
   /**
    * 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. *
@@ -399,11 +418,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    *          the feature is enabled if true, disabled otherwise
    * @see #setRanges(JobConf, Collection)
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setAutoAdjustRanges(JobConf job, boolean enableFeature) {
     InputConfigurator.setAutoAdjustRanges(CLASS, job, enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration has auto-adjust ranges enabled.
    * 
@@ -412,11 +433,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return false if the feature is disabled, true otherwise
    * @since 1.5.0
    * @see #setAutoAdjustRanges(JobConf, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean getAutoAdjustRanges(JobConf job) {
     return InputConfigurator.getAutoAdjustRanges(CLASS, job);
   }
-
+  
   /**
    * Controls the use of the {@link IsolatedScanner} in this job.
    * 
@@ -428,11 +451,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setScanIsolation(JobConf job, boolean enableFeature) {
     InputConfigurator.setScanIsolation(CLASS, job, enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration has isolation enabled.
    * 
@@ -441,11 +466,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setScanIsolation(JobConf, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean isIsolated(JobConf job) {
     return InputConfigurator.isIsolated(CLASS, job);
   }
-
+  
   /**
    * 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.
@@ -458,11 +485,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setLocalIterators(JobConf job, boolean enableFeature) {
     InputConfigurator.setLocalIterators(CLASS, job, enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration uses local iterators.
    * 
@@ -471,11 +500,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setLocalIterators(JobConf, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean usesLocalIterators(JobConf job) {
     return InputConfigurator.usesLocalIterators(CLASS, job);
   }
-
+  
   /**
    * <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
@@ -506,11 +537,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setOfflineTableScan(JobConf job, boolean enableFeature) {
     InputConfigurator.setOfflineTableScan(CLASS, job, enableFeature);
   }
-
+  
   /**
    * Determines whether a configuration has the offline table scan feature enabled.
    * 
@@ -519,11 +552,61 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setOfflineTableScan(JobConf, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean isOfflineScan(JobConf job) {
     return InputConfigurator.isOfflineScan(CLASS, job);
   }
-
+  
+  /**
+   * Sets the {@link TableQueryConfig} objects on the given Hadoop configuration
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param configs
+   *          the table query configs to be set on the configuration.
+   * @since 1.6.0
+   */
+  public static void setTableQueryConfigs(JobConf job, TableQueryConfig... configs) {
+    checkNotNull(configs);
+    InputConfigurator.setTableQueryConfigs(CLASS, job, configs);
+  }
+  
+  /**
+   * Fetches all {@link TableQueryConfig}s that have been set on the given Hadoop configuration.
+   * 
+   * <p>
+   * Note this also returns the {@link TableQueryConfig} representing the table configurations set through the single table input methods (
+   * {@link #setInputTableName(JobConf, String)}, {@link #setRanges(JobConf, java.util.Collection)}, {@link #fetchColumns(JobConf, java.util.Collection)},
+   * {@link #addIterator(JobConf, IteratorSetting)}, etc...)
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @return
+   * @since 1.6.0
+   */
+  public static List<TableQueryConfig> getTableQueryConfigs(JobConf job) {
+    return InputConfigurator.getTableQueryConfigs(CLASS, job);
+  }
+  
+  /**
+   * Fetches a {@link TableQueryConfig} that has been set on the configuration for a specific table.
+   * 
+   * <p>
+   * null is returned in the event that the table doesn't exist.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param tableName
+   *          the table name for which to grab the config object
+   * @return the {@link TableQueryConfig} for the given table
+   * @since 1.6.0
+   */
+  public static TableQueryConfig getTableQueryConfig(JobConf job, String tableName) {
+    return InputConfigurator.getTableQueryConfig(CLASS, job, tableName);
+  }
+  
   /**
    * Initializes an Accumulo {@link TabletLocator} based on the configuration.
    * 
@@ -537,10 +620,10 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static TabletLocator getTabletLocator(JobConf job, String tableName) throws TableNotFoundException {
     return InputConfigurator.getTabletLocator(CLASS, job, tableName);
   }
-
+  
   // 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}.
+   * Check whether a configuration is fully configured to be used with an Accumulo {@link InputFormat}.
    * 
    * @param job
    *          the Hadoop context for the configured job
@@ -551,7 +634,7 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   protected static void validateOptions(JobConf job) throws IOException {
     InputConfigurator.validateOptions(CLASS, job);
   }
-
+  
   /**
    * 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.
@@ -566,7 +649,7 @@ public abstract class InputFormatBase<K,V> implements 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.
      * 
@@ -575,13 +658,13 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
      * @param scanner
      *          the scanner to configure
      */
-    protected void setupIterators(JobConf job, Scanner scanner) {
-      List<IteratorSetting> iterators = getIterators(job);
-      for (IteratorSetting iterator : iterators) {
+    protected void setupIterators(JobConf job, Scanner scanner, String tableName) {
+      TableQueryConfig config = getTableQueryConfig(job, 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.
      */
@@ -593,7 +676,9 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
       String user = getPrincipal(job);
       AuthenticationToken token = getAuthenticationToken(job);
       Authorizations authorizations = getScanAuthorizations(job);
-    
+      
+      TableQueryConfig tableConfig = getTableQueryConfig(job, 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 = split.getTableName();
@@ -606,32 +691,32 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
           throw new IOException("The specified table was not found for id=" + split.getTableId());
         }
       }
-    
+      
       try {
         log.debug("Creating connector with user: " + user);
         Connector conn = instance.getConnector(user, token);
         log.debug("Creating scanner for table: " + getInputTableName(job));
         log.debug("Authorizations are: " + authorizations);
-        if (isOfflineScan(job)) {
+        if (tableConfig.isOfflineScan()) {
           scanner = new OfflineScanner(instance, new Credentials(user, token), split.getTableId(), authorizations);
         } else {
           scanner = conn.createScanner(actualNameForId, authorizations);
         }
-        if (isIsolated(job)) {
+        if (tableConfig.shouldUseIsolatedScanners()) {
           log.info("Creating isolated scanner");
           scanner = new IsolatedScanner(scanner);
         }
-        if (usesLocalIterators(job)) {
+        if (tableConfig.shouldUseLocalIterators()) {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
-        setupIterators(job, scanner);
+        setupIterators(job, scanner, split.getTableName());
       } catch (Exception e) {
         throw new IOException(e);
       }
-    
+      
       // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : getFetchedColumns(job)) {
+      for (Pair<Text,Text> c : tableConfig.getFetchedColumns()) {
         if (c.getSecond() != null) {
           log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
           scanner.fetchColumn(c.getFirst(), c.getSecond());
@@ -640,58 +725,58 @@ public abstract class InputFormatBase<K,V> implements 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 long getPos() throws IOException {
       return numKeysRead;
     }
-  
+    
     @Override
     public float getProgress() throws IOException {
       if (numKeysRead > 0 && currentKey == null)
         return 1.0f;
       return split.getProgress(currentKey);
     }
-  
+    
     protected Key currentKey = null;
-  
+    
   }
-
+  
   Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobConf job, 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(job);
     Connector conn = instance.getConnector(getPrincipal(job), getAuthenticationToken(job));
     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);
@@ -699,73 +784,73 @@ public abstract class InputFormatBase<K,V> implements 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()) {
         Iterator<Entry<Key,Value>> row = rowIter.next();
         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;
   }
-
+  
   /**
    * Read the metadata table to get tablets and match up ranges to them.
    */
@@ -773,112 +858,111 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
     log.setLevel(getLogLevel(job));
     validateOptions(job);
-  
-    boolean autoAdjust = getAutoAdjustRanges(job);
-    List<Range> tablesRanges = getRanges(job);
+    
     LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
-  
-    String tableName = getInputTableName(job);
-    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tablesRanges) : tablesRanges;
-  
-    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;
-    String tableId = null;
-    try {
-      if (isOfflineScan(job)) {
-        binnedRanges = binOfflineTable(job, tableName, ranges);
-        while (binnedRanges == null) {
-          // Some tablets were still online, try again
-          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
-          binnedRanges = binOfflineTable(job, tableName, ranges);
-        }
-      } else {
-        Instance instance = getInstance(job);
-        tl = getTabletLocator(job, tableName);
-        // 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(job), AuthenticationTokenSerializer.deserialize(getTokenClass(job), getToken(job)));
+    List<TableQueryConfig> tableConfigs = getTableQueryConfigs(job);
+    for (TableQueryConfig tableConfig : tableConfigs) {
       
-        while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
-          if (!(instance instanceof MockInstance)) {
-            tableId = Tables.getTableId(instance, tableName);
-            if (!Tables.exists(instance, tableId))
-              throw new TableDeletedException(tableId);
-            if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-              throw new TableOfflineException(instance, tableId);
+      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;
+      try {
+        if (tableConfig.isOfflineScan()) {
+          binnedRanges = binOfflineTable(job, tableConfig.getTableName(), ranges);
+          while (binnedRanges == null) {
+            // Some tablets were still online, try again
+            UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
+            binnedRanges = binOfflineTable(job, tableConfig.getTableName(), ranges);
           }
-          binnedRanges.clear();
-          log.warn("Unable to locate bins for specified ranges. Retrying.");
-          UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
+        } else {
+          Instance instance = getInstance(job);
+          tl = getTabletLocator(job, tableConfig.getTableName());
+          // 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(job), getAuthenticationToken(job));
+          
+          while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
+            if (!(instance instanceof MockInstance)) {
+              if (!Tables.exists(instance, tableId))
+                throw new TableDeletedException(tableId);
+              if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+                throw new TableOfflineException(instance, tableId);
+              tableId = Tables.getTableId(instance, tableConfig.getTableName());
+            }
+            binnedRanges.clear();
+            log.warn("Unable to locate bins for specified ranges. Retrying.");
+            UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
+            tl.invalidateCache();
+          }
         }
+      } catch (Exception e) {
+        throw new IOException(e);
       }
-    } 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];
-      String location = hostNameCache.get(ip);
-      if (location == null) {
-        InetAddress inetAddress = InetAddress.getByName(ip);
-        location = inetAddress.getHostName();
-        hostNameCache.put(ip, location);
-      }
-    
-      for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-        Range ke = extentRanges.getKey().toDataRange();
-        for (Range r : extentRanges.getValue()) {
-          if (autoAdjust) {
-            // divide ranges into smaller ranges, based on the tablets
-            splits.add(new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location}));
-          } else {
-            // don't divide ranges
-            ArrayList<String> locations = splitsToAdd.get(r);
-            if (locations == null)
-              locations = new ArrayList<String>(1);
-            locations.add(location);
-            splitsToAdd.put(r, locations);
+      
+      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];
+        String location = hostNameCache.get(ip);
+        if (location == null) {
+          InetAddress inetAddress = InetAddress.getByName(ip);
+          location = inetAddress.getHostName();
+          hostNameCache.put(ip, location);
+        }
+        for (Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
+          Range ke = extentRanges.getKey().toDataRange();
+          for (Range r : extentRanges.getValue()) {
+            if (autoAdjust) {
+              // divide ranges into smaller ranges, based on the tablets
+              splits.add(new RangeInputSplit(tableConfig.getTableName(), tableId, ke.clip(r), new String[] {location}));
+            } else {
+              // don't divide ranges
+              ArrayList<String> locations = splitsToAdd.get(r);
+              if (locations == null)
+                locations = new ArrayList<String>(1);
+              locations.add(location);
+              splitsToAdd.put(r, locations);
+            }
           }
         }
       }
+      
+      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])));
     }
-  
-    if (!autoAdjust)
-      for (Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet())
-        splits.add(new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0])));
-  
+    
     return splits.toArray(new InputSplit[splits.size()]);
   }
-
+  
   /**
    * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
    */
   public static class RangeInputSplit extends org.apache.accumulo.core.client.mapreduce.InputFormatBase.RangeInputSplit implements InputSplit {
-  
+    
     public RangeInputSplit() {
       super();
     }
-  
+    
     public RangeInputSplit(RangeInputSplit split) throws IOException {
       super(split);
     }
-  
+    
     protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
       super(table, tableId, range, locations);
     }
   }
-
+  
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 1cbb606..44dfc33 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -39,9 +39,9 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * <ul>
  * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
  * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloInputFormat#setInputTableName(Job, String)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
  * <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, String, String)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
+ * <li>{@link AccumuloInputFormat#setTableQueryConfigs(Job, org.apache.accumulo.core.conf.TableQueryConfig...)} 
  * </ul>
  * 
  * Other static methods are optional.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/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 2cec670..acff7e2 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
@@ -168,7 +168,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return the table name
    * @since 1.5.0
    * @see #setInputTableName(Job, String)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static String getInputTableName(JobContext context) {
     return InputConfigurator.getInputTableName(CLASS, getConfiguration(context));
   }
@@ -285,7 +287,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param tableName
    *          the table to use when the tablename is null in the write call
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setInputTableName(Job job, String tableName) {
     InputConfigurator.setInputTableName(CLASS, job.getConfiguration(), tableName);
   }
@@ -317,15 +321,16 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
   }
 
   /**
-   * 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.Collection)}
+   * Sets the input ranges to scan for all tables associated with this job.
    * 
    * @param job
    *          the Hadoop job instance to be configured
    * @param ranges
    *          the ranges that will be mapped over
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setRanges(Job job, Collection<Range> ranges) {
     InputConfigurator.setRanges(CLASS, job.getConfiguration(), ranges);
   }
@@ -339,7 +344,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @since 1.5.0
    * @see #setRanges(Job, Collection)
    * @see #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Collection)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static List<Range> getRanges(JobContext context) throws IOException {
     return InputConfigurator.getRanges(CLASS, getConfiguration(context));
   }
@@ -353,7 +360,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    *          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.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void fetchColumns(Job job, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     InputConfigurator.fetchColumns(CLASS, job.getConfiguration(), columnFamilyColumnQualifierPairs);
   }
@@ -366,7 +375,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return a set of columns
    * @since 1.5.0
    * @see #fetchColumns(Job, Collection)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static Set<Pair<Text,Text>> getFetchedColumns(JobContext context) {
     return InputConfigurator.getFetchedColumns(CLASS, getConfiguration(context));
   }
@@ -379,7 +390,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param cfg
    *          the configuration of the iterator
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void addIterator(Job job, IteratorSetting cfg) {
     InputConfigurator.addIterator(CLASS, job.getConfiguration(), cfg);
   }
@@ -392,7 +405,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return a list of iterators
    * @since 1.5.0
    * @see #addIterator(Job, IteratorSetting)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static List<IteratorSetting> getIterators(JobContext context) {
     return InputConfigurator.getIterators(CLASS, getConfiguration(context));
   }
@@ -410,7 +425,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    *          the feature is enabled if true, disabled otherwise
    * @see #setRanges(Job, Collection)
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setAutoAdjustRanges(Job job, boolean enableFeature) {
     InputConfigurator.setAutoAdjustRanges(CLASS, job.getConfiguration(), enableFeature);
   }
@@ -423,7 +440,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return false if the feature is disabled, true otherwise
    * @since 1.5.0
    * @see #setAutoAdjustRanges(Job, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean getAutoAdjustRanges(JobContext context) {
     return InputConfigurator.getAutoAdjustRanges(CLASS, getConfiguration(context));
   }
@@ -435,8 +454,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    *          the Hadoop job instance to be configured
    * @param configs
    *          the table query configs to be set on the configuration.
+   * @since 1.6.0
    */
-  public static void setTableQueryConfigs(JobContext job, TableQueryConfig... configs) {
+  public static void setTableQueryConfigs(Job job, TableQueryConfig... configs) {
     checkNotNull(configs);
     InputConfigurator.setTableQueryConfigs(CLASS, getConfiguration(job), configs);
   }
@@ -445,7 +465,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * Fetches all {@link TableQueryConfig}s that have been set on the given Hadoop configuration.
    * 
    * <p>
-   * Note this also returns the {@link TableQueryConfig} representing the table configurations set through the single table input methods (
+   * Note this also returns the {@link TableQueryConfig} representing the table configurations set through the single table input methods like
    * {@link #setInputTableName(org.apache.hadoop.mapreduce.Job, String)}, {@link #setRanges(org.apache.hadoop.mapreduce.Job, java.util.Collection)},
    * {@link #fetchColumns(org.apache.hadoop.mapreduce.Job, java.util.Collection)},
    * {@link #addIterator(org.apache.hadoop.mapreduce.Job, org.apache.accumulo.core.client.IteratorSetting)}, etc...)
@@ -453,6 +473,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param job
    *          the Hadoop job instance to be configured
    * @return
+   * @since 1.6.0
    */
   public static List<TableQueryConfig> getTableQueryConfigs(JobContext job) {
     return InputConfigurator.getTableQueryConfigs(CLASS, getConfiguration(job));
@@ -469,6 +490,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param tableName
    *          the table name for which to grab the config object
    * @return the {@link TableQueryConfig} for the given table
+   * @since 1.6.0
    */
   public static TableQueryConfig getTableQueryConfig(JobContext job, String tableName) {
     return InputConfigurator.getTableQueryConfig(CLASS,getConfiguration(job),tableName);
@@ -485,7 +507,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setScanIsolation(Job job, boolean enableFeature) {
     InputConfigurator.setScanIsolation(CLASS, job.getConfiguration(), enableFeature);
   }
@@ -498,7 +522,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setScanIsolation(Job, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean isIsolated(JobContext context) {
     return InputConfigurator.isIsolated(CLASS, getConfiguration(context));
   }
@@ -515,7 +541,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setLocalIterators(Job job, boolean enableFeature) {
     InputConfigurator.setLocalIterators(CLASS, job.getConfiguration(), enableFeature);
   }
@@ -528,7 +556,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setLocalIterators(Job, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean usesLocalIterators(JobContext context) {
     return InputConfigurator.usesLocalIterators(CLASS, getConfiguration(context));
   }
@@ -563,7 +593,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setOfflineTableScan(Job job, boolean enableFeature) {
     InputConfigurator.setOfflineTableScan(CLASS, job.getConfiguration(), enableFeature);
   }
@@ -576,7 +608,9 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setOfflineTableScan(Job, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   protected static boolean isOfflineScan(JobContext context) {
     return InputConfigurator.isOfflineScan(CLASS, getConfiguration(context));
   }
@@ -680,16 +714,16 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
         Connector conn = instance.getConnector(principal, token);
         log.debug("Creating scanner for table: " + split.getTableName());
         log.debug("Authorizations are: " + authorizations);
-        if (isOfflineScan(attempt)) {
+        if (tableConfig.isOfflineScan()) {
           scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
         } else {
           scanner = conn.createScanner(actualNameForId, authorizations);
         }
-        if (isIsolated(attempt)) {
+        if (tableConfig.shouldUseIsolatedScanners()) {
           log.info("Creating isolated scanner");
           scanner = new IsolatedScanner(scanner);
         }
-        if (usesLocalIterators(attempt)) {
+        if (tableConfig.shouldUseLocalIterators()) {
           log.info("Using local iterators");
           scanner = new ClientSideIteratorScanner(scanner);
         }
@@ -699,7 +733,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       }
     
       // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : tableConfig.getColumns()) {
+      for (Pair<Text,Text> c : tableConfig.getFetchedColumns()) {
         if (c.getSecond() != null) {
           log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
           scanner.fetchColumn(c.getFirst(), c.getSecond());
@@ -867,7 +901,7 @@ public abstract class InputFormatBase<K,V> extends InputFormat<K,V> {
       Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
       TabletLocator tl;
       try {
-        if (isOfflineScan(conf)) {
+        if (tableConfig.isOfflineScan()) {
           binnedRanges = binOfflineTable(conf, tableConfig.getTableName(), ranges);
           while (binnedRanges == null) {
             // Some tablets were still online, try again

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/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 42097f1..ce021a9 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
@@ -66,7 +66,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * 
    * @since 1.5.0
    */
-  public static enum ScanOpts {
+  public static enum ScanOpts { 
     TABLE_NAME, AUTHORIZATIONS, RANGES, COLUMNS, ITERATORS, TABLE_CONFIGS
   }
   
@@ -74,7 +74,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * Configuration keys for various features.
    * 
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static enum Features {
     AUTO_ADJUST_RANGES, SCAN_ISOLATION, USE_LOCAL_ITERATORS, SCAN_OFFLINE
   }
@@ -89,7 +91,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param tableName
    *          the table to use when the tablename is null in the write call
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setInputTableName(Class<?> implementingClass, Configuration conf, String tableName) {
     notNull(tableName);
     conf.set(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME), tableName);
@@ -103,7 +107,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param conf
    *          the Hadoop configuration object to configure
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static String getInputTableName(Class<?> implementingClass, Configuration conf) {
     return conf.get(enumToConfKey(implementingClass, ScanOpts.TABLE_NAME));
   }
@@ -152,7 +158,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @throws IllegalArgumentException
    *           if the ranges cannot be encoded into base 64
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setRanges(Class<?> implementingClass, Configuration conf, Collection<Range> ranges) {
     notNull(ranges);
     
@@ -181,7 +189,9 @@ public class InputConfigurator extends ConfiguratorBase {
    *           if the ranges have been encoded improperly
    * @since 1.5.0
    * @see #setRanges(Class, Configuration, Collection)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static List<Range> getRanges(Class<?> implementingClass, Configuration conf) throws IOException {
     
     Collection<String> encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES));
@@ -205,7 +215,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return a list of iterators
    * @since 1.5.0
    * @see #addIterator(Class, Configuration, IteratorSetting)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static List<IteratorSetting> getIterators(Class<?> implementingClass, Configuration conf) {
     String iterators = conf.get(enumToConfKey(implementingClass, ScanOpts.ITERATORS));
     
@@ -242,7 +254,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @throws IllegalArgumentException
    *           if the column family is null
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void fetchColumns(Class<?> implementingClass, Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
     notNull(columnFamilyColumnQualifierPairs);
     ArrayList<String> columnStrings = new ArrayList<String>();
@@ -269,7 +283,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return a set of columns
    * @since 1.5.0
    * @see #fetchColumns(Class, Configuration, Collection)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   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))) {
@@ -293,7 +309,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @throws IllegalArgumentException
    *           if the iterator can't be serialized into the configuration
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void addIterator(Class<?> implementingClass, Configuration conf, IteratorSetting cfg) {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     String newIter;
@@ -333,7 +351,9 @@ public class InputConfigurator extends ConfiguratorBase {
    *          the feature is enabled if true, disabled otherwise
    * @see #setRanges(Class, Configuration, Collection)
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setAutoAdjustRanges(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
     conf.setBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), enableFeature);
   }
@@ -348,7 +368,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return false if the feature is disabled, true otherwise
    * @since 1.5.0
    * @see #setAutoAdjustRanges(Class, Configuration, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static Boolean getAutoAdjustRanges(Class<?> implementingClass, Configuration conf) {
     return conf.getBoolean(enumToConfKey(implementingClass, Features.AUTO_ADJUST_RANGES), true);
   }
@@ -366,7 +388,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setScanIsolation(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
     conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), enableFeature);
   }
@@ -381,7 +405,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setScanIsolation(Class, Configuration, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static Boolean isIsolated(Class<?> implementingClass, Configuration conf) {
     return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_ISOLATION), false);
   }
@@ -400,7 +426,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setLocalIterators(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
     conf.setBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), enableFeature);
   }
@@ -415,7 +443,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setLocalIterators(Class, Configuration, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static Boolean usesLocalIterators(Class<?> implementingClass, Configuration conf) {
     return conf.getBoolean(enumToConfKey(implementingClass, Features.USE_LOCAL_ITERATORS), false);
   }
@@ -452,7 +482,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param enableFeature
    *          the feature is enabled if true, disabled otherwise
    * @since 1.5.0
+   * @deprecated since 1.6.0
    */
+  @Deprecated
   public static void setOfflineTableScan(Class<?> implementingClass, Configuration conf, boolean enableFeature) {
     conf.setBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), enableFeature);
   }
@@ -467,7 +499,9 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return true if the feature is enabled, false otherwise
    * @since 1.5.0
    * @see #setOfflineTableScan(Class, Configuration, boolean)
+   * @deprecated since 1.6.0
    */
+  @Deprecated 
   public static Boolean isOfflineScan(Class<?> implementingClass, Configuration conf) {
     return conf.getBoolean(enumToConfKey(implementingClass, Features.SCAN_OFFLINE), false);
   }
@@ -481,6 +515,7 @@ public class InputConfigurator extends ConfiguratorBase {
    *          the Hadoop configuration object to configure
    * @param tconf
    *          an array of {@link TableQueryConfig} objects to associate with the job
+   * @since 1.6.0
    */
   public static void setTableQueryConfigs(Class<?> implementingClass, Configuration conf, TableQueryConfig... tconf) {
     List<String> tableQueryConfigStrings = new ArrayList<String>();
@@ -505,6 +540,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * @param conf
    *          the Hadoop configuration object to configure
    * @return all of the table query configs for the job
+   * @since 1.6.0
    */
   public static List<TableQueryConfig> getTableQueryConfigs(Class<?> implementingClass, Configuration conf) {
     List<TableQueryConfig> configs = new ArrayList<TableQueryConfig>();
@@ -543,6 +579,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * @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 TableQueryConfig getTableQueryConfig(Class<?> implementingClass, Configuration conf, String tableName) {
     List<TableQueryConfig> queryConfigs = getTableQueryConfigs(implementingClass, conf);
@@ -566,7 +603,7 @@ public class InputConfigurator extends ConfiguratorBase {
    * @return an Accumulo tablet locator
    * @throws TableNotFoundException
    *           if the table name set on the configuration doesn't exist
-   * @since 1.6.0
+   * @since 1.5.0
    */
   public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf, String tableName) throws TableNotFoundException {
     String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
@@ -636,6 +673,7 @@ public class InputConfigurator extends ConfiguratorBase {
    *          the Hadoop instance for which to retrieve the configuration
    * @return the config object built from the single input table properties set on the job
    * @throws IOException
+   * @since 1.6.0
    */
   protected static TableQueryConfig getDefaultTableConfig(Class<?> implementingClass, Configuration conf) throws IOException {
     String tableName = getInputTableName(implementingClass, conf);
@@ -646,7 +684,7 @@ public class InputConfigurator extends ConfiguratorBase {
         queryConfig.setIterators(itrs);
       Set<Pair<Text,Text>> columns = getFetchedColumns(implementingClass, conf);
       if (columns != null)
-        queryConfig.setColumns(columns);
+        queryConfig.fetchColumns(columns);
       List<Range> ranges = getRanges(implementingClass, conf);
       if (ranges != null)
         queryConfig.setRanges(ranges);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/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 26feca3..a5278fa 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
@@ -32,6 +32,9 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
+/**
+ * This class to holds a query configuration for a table. It contains all the properties needed to specify how rows should be returned from the table.
+ */
 public class TableQueryConfig implements Writable {
   
   private String tableName;
@@ -53,69 +56,195 @@ public class TableQueryConfig implements Writable {
     readFields(input);
   }
   
+  /**
+   * Returns the table name associated with this configuration
+   */
+  public String getTableName() {
+    return tableName;
+  }
+  
+  /**
+   * 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
+   * 
+   * @param ranges
+   *          the ranges that will be mapped over
+   * @since 1.6.0
+   */
   public TableQueryConfig setRanges(List<Range> ranges) {
     this.ranges = ranges;
     return this;
   }
   
-  public TableQueryConfig setColumns(Set<Pair<Text,Text>> columns) {
+  /**
+   * Returns the ranges to be queried in the configuration
+   */
+  public List<Range> getRanges() {
+    return ranges != null ? ranges : new ArrayList<Range>();
+  }
+  
+  /**
+   * Restricts the columns that will be mapped over for this job for the default input table.
+   * 
+   * @param columns
+   *          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
+   */
+  public TableQueryConfig fetchColumns(Set<Pair<Text,Text>> columns) {
     this.columns = columns;
     return this;
   }
   
+  /**
+   * Returns the columns to be fetched for this configuration
+   */
+  public Set<Pair<Text,Text>> getFetchedColumns() {
+    return columns != null ? columns : new HashSet<Pair<Text,Text>>();
+  }
+  
+  /**
+   * Set iterators on to be used in the query.
+   * 
+   * @param iterators
+   *          the configurations for the iterators
+   * @since 1.6.0
+   */
   public TableQueryConfig setIterators(List<IteratorSetting> iterators) {
     this.iterators = iterators;
     return this;
   }
   
+  /**
+   * Returns the iterators to be set on this configuration
+   */
+  public List<IteratorSetting> getIterators() {
+    return iterators != null ? iterators : new ArrayList<IteratorSetting>();
+  }
+  
+  /**
+   * 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 autoAdjustRanges
+   *          the feature is enabled if true, disabled otherwise
+   * @see #setRanges(java.util.List)
+   * @since 1.6.0
+   */
   public TableQueryConfig setAutoAdjustRanges(boolean autoAdjustRanges) {
     this.autoAdjustRanges = autoAdjustRanges;
     return this;
   }
   
-  public TableQueryConfig setUseLocalIterators(boolean useLocalIterators) {
-    this.useLocalIterators = useLocalIterators;
-    return this;
+  /**
+   * Determines whether a configuration has auto-adjust ranges enabled.
+   * 
+   * @return false if the feature is disabled, true otherwise
+   * @since 1.6.0
+   * @see #setAutoAdjustRanges(boolean)
+   */
+  public boolean shouldAutoAdjustRanges() {
+    return autoAdjustRanges;
   }
   
-  public TableQueryConfig setUseIsolatedScanners(boolean useIsolatedScanners) {
-    this.useIsolatedScanners = useIsolatedScanners;
+  /**
+   * Controls the use of the {@link org.apache.accumulo.core.client.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 useLocalIterators
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public TableQueryConfig setUseLocalIterators(boolean useLocalIterators) {
+    this.useLocalIterators = useLocalIterators;
     return this;
   }
   
-  public boolean isOfflineScan() {
-    return offlineScan;
+  /**
+   * Determines whether a configuration uses local iterators.
+   * 
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setUseLocalIterators(boolean)
+   * @deprecated since 1.6.0
+   */
+  public boolean shouldUseLocalIterators() {
+    return useLocalIterators;
   }
   
+  /**
+   * <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. The accumulo-site.xml may need to be on the mapper's classpath if HDFS or the Accumulo directory in HDFS are non-standard.
+   * 
+   * <p>
+   * One way to use this feature is to clone a table, take the clone offline, and use the clone as the input table for a map reduce job. If you plan to map
+   * reduce over the data many times, it may be better to the compact the table, clone it, take it offline, and use the clone for all map reduce jobs. The
+   * reason to do this is that compaction will reduce each tablet in the table to one file, and it is faster to read from one file.
+   * 
+   * <p>
+   * There are two possible advantages to reading a tables file directly out of HDFS. First, you may see better read performance. Second, it will support
+   * speculative execution better. When reading an online table speculative execution can put more load on an already slow tablet server.
+   * 
+   * <p>
+   * By default, this feature is <b>disabled</b>.
+   * 
+   * @param offlineScan
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
   public TableQueryConfig setOfflineScan(boolean offlineScan) {
     this.offlineScan = offlineScan;
     return this;
   }
   
-  public String getTableName() {
-    return tableName;
-  }
-  
-  public List<IteratorSetting> getIterators() {
-    return iterators != null ? iterators : new ArrayList<IteratorSetting>();
-  }
-  
-  public List<Range> getRanges() {
-    return ranges != null ? ranges : new ArrayList<Range>();
-  }
-  
-  public Set<Pair<Text,Text>> getColumns() {
-    return columns != null ? columns : new HashSet<Pair<Text,Text>>();
-  }
-  
-  public boolean shouldAutoAdjustRanges() {
-    return autoAdjustRanges;
+  /**
+   * Determines whether a configuration has the offline table scan feature enabled.
+   * 
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setOfflineScan(boolean)
+   */
+  public boolean isOfflineScan() {
+    return offlineScan;
   }
   
-  public boolean shouldUseLocalIterators() {
-    return useLocalIterators;
+  /**
+   * Controls the use of the {@link org.apache.accumulo.core.client.IsolatedScanner} in this job.
+   * 
+   * <p>
+   * By default, this feature is <b>disabled</b>.
+   * 
+   * @param useIsolatedScanners
+   *          the feature is enabled if true, disabled otherwise
+   * @since 1.6.0
+   */
+  public TableQueryConfig setUseIsolatedScanners(boolean useIsolatedScanners) {
+    this.useIsolatedScanners = useIsolatedScanners;
+    return this;
   }
   
+  /**
+   * Determines whether a configuration has isolation enabled.
+   * 
+   * @return true if the feature is enabled, false otherwise
+   * @since 1.6.0
+   * @see #setUseIsolatedScanners(boolean)
+   */
   public boolean shouldUseIsolatedScanners() {
     return useIsolatedScanners;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
index a94d80f..d440a6c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -31,12 +32,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.TableQueryConfig;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
@@ -55,6 +59,7 @@ public class AccumuloInputFormatTest {
   private static final String PREFIX = AccumuloInputFormatTest.class.getSimpleName();
   private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
   private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
+  private static final String TEST_TABLE_2 = PREFIX + "_mapred_table_2";
   
   /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
@@ -203,10 +208,11 @@ public class AccumuloInputFormatTest {
       @Override
       public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
         try {
+          String tableName = ((InputFormatBase.RangeInputSplit) reporter.getInputSplit()).getTableName();
           if (key != null)
             assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
+          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
+          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
         } catch (AssertionError e) {
           e1 = e;
         }
@@ -231,13 +237,14 @@ public class AccumuloInputFormatTest {
     @Override
     public int run(String[] args) throws Exception {
       
-      if (args.length != 3) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
+      if (args.length != 4) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
       }
       
       String user = args[0];
       String pass = args[1];
-      String table = args[2];
+      String table1 = args[2];
+      String table2 = args[3];
       
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
@@ -245,9 +252,13 @@ public class AccumuloInputFormatTest {
       job.setInputFormat(AccumuloInputFormat.class);
       
       AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
       
+      TableQueryConfig tableConfig1 = new TableQueryConfig(table1);
+      TableQueryConfig tableConfig2 = new TableQueryConfig(table2);
+      
+      AccumuloInputFormat.setTableQueryConfigs(job, tableConfig1, tableConfig2);
+      
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
@@ -268,16 +279,70 @@ public class AccumuloInputFormatTest {
     MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
+    c.tableOperations().create(TEST_TABLE_2);
     BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
+    BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
+      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
+      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
+      bw.addMutation(t1m);
+      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
+      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
+      bw2.addMutation(t2m);
     }
     bw.close();
+    bw2.close();
     
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1});
+    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
     assertNull(e1);
     assertNull(e2);
   }
+  
+  /**
+   * Verify {@link org.apache.accumulo.core.conf.TableQueryConfig} objects get correctly serialized in the JobContext.
+   */
+  @Test
+  public void testTableQueryConfigSerialization() throws IOException {
+    
+    JobConf job = new JobConf();
+    
+    TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
+        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+    
+    TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
+        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+    
+    AccumuloInputFormat.setTableQueryConfigs(job, table1, table2);
+    
+    assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
+    assertEquals(table2, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
+  }
+  
+  /**
+   * Verify that union of legacy input and new multi-table input get returned for backwards compatibility.
+   */
+  @Test
+  public void testTableQueryConfigSingleAndMultitableMethods() throws IOException {
+    
+    JobConf job = new JobConf();
+    
+    TableQueryConfig table1 = new TableQueryConfig(TEST_TABLE_1).setRanges(Collections.singletonList(new Range("a", "b")))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
+        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+    
+    TableQueryConfig table2 = new TableQueryConfig(TEST_TABLE_2).setRanges(Collections.singletonList(new Range("a", "b")))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
+        .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
+    
+    AccumuloInputFormat.setTableQueryConfigs(job, table1);
+    AccumuloInputFormat.setInputTableName(job, table2.getTableName());
+    AccumuloInputFormat.setRanges(job, table2.getRanges());
+    AccumuloInputFormat.fetchColumns(job, table2.getFetchedColumns());
+    AccumuloInputFormat.addIterator(job, table2.getIterators().get(0));
+    
+    assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
+    assertEquals(table2, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/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 e7603c4..6f92dec 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
@@ -235,7 +235,7 @@ public class AccumuloInputFormatTest {
     public int run(String[] args) throws Exception {
       
       if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
       }
       
       String user = args[0];
@@ -250,7 +250,6 @@ public class AccumuloInputFormatTest {
       
       AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       
-      // AccumuloInputFormat.setInputTableNames(job, Arrays.asList(new String[]{table1, table2}));
       TableQueryConfig tableConfig1 = new TableQueryConfig(table1);
       TableQueryConfig tableConfig2 = new TableQueryConfig(table2);
       
@@ -310,11 +309,11 @@ public class AccumuloInputFormatTest {
     Job job = new Job();
     
     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"))))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
     
     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"))))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
     
     AccumuloInputFormat.setTableQueryConfigs(job, table1, table2);
@@ -332,17 +331,17 @@ public class AccumuloInputFormatTest {
     Job job = new Job();
     
     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"))))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
     
     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"))))
+        .fetchColumns(Collections.singleton(new Pair<Text, Text>(new Text("CF1"), new Text("CQ1"))))
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
     
     AccumuloInputFormat.setTableQueryConfigs(job, table1);
     AccumuloInputFormat.setInputTableName(job, table2.getTableName());
     AccumuloInputFormat.setRanges(job, table2.getRanges());
-    AccumuloInputFormat.fetchColumns(job, table2.getColumns());
+    AccumuloInputFormat.fetchColumns(job, table2.getFetchedColumns());
     AccumuloInputFormat.addIterator(job, table2.getIterators().get(0));
     
     assertEquals(table1, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3227a822/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java b/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
index 3c26a07..ac3340f 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/TableQueryConfigTest.java
@@ -55,12 +55,12 @@ public class TableQueryConfigTest {
     Set<Pair<Text,Text>> columns = new HashSet<Pair<Text,Text>>();
     columns.add(new Pair<Text,Text>(new Text("cf1"), new Text("cq1")));
     columns.add(new Pair<Text,Text>(new Text("cf2"), null));
-    tableQueryConfig.setColumns(columns);
+    tableQueryConfig.fetchColumns(columns);
     
     byte[] serialized = serialize(tableQueryConfig);
     TableQueryConfig actualConfig = deserialize(serialized);
     
-    assertEquals(actualConfig.getColumns(), columns);
+    assertEquals(actualConfig.getFetchedColumns(), columns);
   }
   
   @Test