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/16 03:30:31 UTC

[2/2] git commit: ACCUMULO-391 Performing refactor to the legacy mapred InputFormatBase

ACCUMULO-391 Performing refactor to the legacy mapred InputFormatBase


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

Branch: refs/heads/master
Commit: cb97e82a4283ef028cd0500b1c23941386ecb6f8
Parents: 9a63ff4
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Tue Oct 15 21:20:42 2013 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Tue Oct 15 21:20:42 2013 -0400

----------------------------------------------------------------------
 .../core/client/mapred/AbstractInputFormat.java | 649 +++++++++++++++++++
 .../mapred/AccumuloMultiTableInputFormat.java   |  80 +++
 .../core/client/mapred/InputFormatBase.java     | 643 +-----------------
 .../client/mapreduce/AbstractInputFormat.java   |  29 +-
 .../AccumuloMultiTableInputFormat.java          |  16 +
 .../client/mapred/AccumuloInputFormatTest.java  | 178 ++---
 .../AccumuloMultiTableInputFormatTest.java      | 188 ++++++
 .../AccumuloMultiTableInputFormatTest.java      |  98 +--
 8 files changed, 1077 insertions(+), 804 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cb97e82a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
new file mode 100644
index 0000000..ba1dc6a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -0,0 +1,649 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapred;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientSideIteratorScanner;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IsolatedScanner;
+import org.apache.accumulo.core.client.RowIterator;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableDeletedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.impl.OfflineScanner;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
+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.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
+  protected 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.
+   * 
+   * <p>
+   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
+   * conversion to a string, and is not intended to be secure.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param principal
+   *          a valid Accumulo user name (user must have Table.CREATE permission)
+   * @param token
+   *          the user's password
+   * @throws org.apache.accumulo.core.client.AccumuloSecurityException
+   * @since 1.5.0
+   */
+  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.
+   * 
+   * <p>
+   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt to be more secure than storing it in the Configuration.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param principal
+   *          a valid Accumulo user name (user must have Table.CREATE permission)
+   * @param tokenFile
+   *          the path to the token file
+   * @throws AccumuloSecurityException
+   * @since 1.6.0
+   */
+  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.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return true if the connector has been configured, false otherwise
+   * @since 1.5.0
+   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
+   */
+  protected static Boolean isConnectorInfoSet(JobConf job) {
+    return InputConfigurator.isConnectorInfoSet(CLASS, job);
+  }
+
+  /**
+   * Gets the user name from the configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the user name
+   * @since 1.5.0
+   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
+   */
+  protected static String getPrincipal(JobConf job) {
+    return InputConfigurator.getPrincipal(CLASS, job);
+  }
+
+  /**
+   * Gets the serialized token class from either the configuration or the token file.
+   * 
+   * @since 1.5.0
+   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
+   */
+  @Deprecated
+  protected static String getTokenClass(JobConf job) {
+    return getAuthenticationToken(job).getClass().getName();
+  }
+
+  /**
+   * Gets the serialized token from either the configuration or the token file.
+   * 
+   * @since 1.5.0
+   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
+   */
+  @Deprecated
+  protected static byte[] getToken(JobConf job) {
+    return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
+  }
+
+  /**
+   * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the principal's authentication token
+   * @since 1.6.0
+   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
+   * @see #setConnectorInfo(JobConf, String, String)
+   */
+  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
+    return InputConfigurator.getAuthenticationToken(CLASS, job);
+  }
+
+  /**
+   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param instanceName
+   *          the Accumulo instance name
+   * @param zooKeepers
+   *          a comma-separated list of zookeeper servers
+   * @since 1.5.0
+   */
+  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
+    InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
+  }
+
+  /**
+   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param instanceName
+   *          the Accumulo instance name
+   * @since 1.5.0
+   */
+  public static void setMockInstance(JobConf job, String instanceName) {
+    InputConfigurator.setMockInstance(CLASS, job, instanceName);
+  }
+
+  /**
+   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return an Accumulo instance
+   * @since 1.5.0
+   * @see #setZooKeeperInstance(JobConf, String, String)
+   * @see #setMockInstance(JobConf, String)
+   */
+  protected static Instance getInstance(JobConf job) {
+    return InputConfigurator.getInstance(CLASS, job);
+  }
+
+  /**
+   * Sets the log level for this job.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @param level
+   *          the logging level
+   * @since 1.5.0
+   */
+  public static void setLogLevel(JobConf job, Level level) {
+    InputConfigurator.setLogLevel(CLASS, job, level);
+  }
+
+  /**
+   * Gets the log level from this configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the log level
+   * @since 1.5.0
+   * @see #setLogLevel(JobConf, Level)
+   */
+  protected static Level getLogLevel(JobConf job) {
+    return InputConfigurator.getLogLevel(CLASS, job);
+  }
+
+  /**
+   * Sets the {@link org.apache.accumulo.core.security.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
+   *          the user's authorizations
+   * @since 1.5.0
+   */
+  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
+    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
+  }
+
+  /**
+   * Gets the authorizations to set for the scans from the configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return the Accumulo scan authorizations
+   * @since 1.5.0
+   * @see #setScanAuthorizations(JobConf, Authorizations)
+   */
+  protected static Authorizations getScanAuthorizations(JobConf job) {
+    return InputConfigurator.getScanAuthorizations(CLASS, job);
+  }
+
+  /**
+   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @return an Accumulo tablet locator
+   * @throws org.apache.accumulo.core.client.TableNotFoundException
+   *           if the table name set on the configuration doesn't exist
+   * @since 1.5.0
+   */
+  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 InputFormat}.
+   * 
+   * @param job
+   *          the Hadoop context for the configured job
+   * @throws java.io.IOException
+   *           if the context is improperly configured
+   * @since 1.5.0
+   */
+  protected static void validateOptions(JobConf job) throws IOException {
+    InputConfigurator.validateOptions(CLASS, job);
+  }
+
+  /**
+   * Fetches all {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig}s that have been set on the given Hadoop configuration.
+   * 
+   * @param job
+   *          the Hadoop job instance to be configured
+   * @return
+   * @since 1.6.0
+   */
+  public static Map<String,BatchScanConfig> getBatchScanConfigs(JobConf job) {
+    return InputConfigurator.getBatchScanConfigs(CLASS, job);
+  }
+
+  /**
+   * Fetches a {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} 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 org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the given table
+   * @since 1.6.0
+   */
+  public static BatchScanConfig getBatchScanConfig(JobConf job, String tableName) {
+    return InputConfigurator.getTableQueryConfig(CLASS, job, tableName);
+  }
+
+  /**
+   * An abstract base class to be used to create {@link org.apache.hadoop.mapred.RecordReader} instances that convert from Accumulo
+   * {@link org.apache.accumulo.core.data.Key}/{@link org.apache.accumulo.core.data.Value} pairs to the user's K/V types.
+   * 
+   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to update the following variables:
+   * <ul>
+   * <li>Key {@link #currentKey} (used for progress reporting)</li>
+   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
+   * </ul>
+   */
+  protected abstract static class AbstractRecordReader<K,V> implements RecordReader<K,V> {
+    protected long numKeysRead;
+    protected Iterator<Map.Entry<Key,Value>> scannerIterator;
+    protected RangeInputSplit split;
+
+    /**
+     * Configures the iterators on a scanner for the given table name.
+     * 
+     * @param job
+     *          the Hadoop job configuration
+     * @param scanner
+     *          the scanner for which to configure the iterators
+     * @param tableName
+     *          the table name for which the scanner is configured
+     */
+    protected abstract void setupIterators(JobConf job, Scanner scanner, String tableName);
+
+    /**
+     * Initialize a scanner over the given input split using this task attempt configuration.
+     */
+    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
+      Scanner scanner;
+      split = (RangeInputSplit) inSplit;
+      log.debug("Initializing input split: " + split.getRange());
+      Instance instance = getInstance(job);
+      String user = getPrincipal(job);
+      AuthenticationToken token = getAuthenticationToken(job);
+      Authorizations authorizations = getScanAuthorizations(job);
+
+      BatchScanConfig tableConfig = getBatchScanConfig(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();
+      if (!(instance instanceof MockInstance)) {
+        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());
+        }
+      }
+
+      try {
+        log.debug("Creating connector with user: " + user);
+        Connector conn = instance.getConnector(user, token);
+        log.debug("Creating scanner for table: " + split.getTableName());
+        log.debug("Authorizations are: " + authorizations);
+        if (tableConfig.isOfflineScan()) {
+          scanner = new OfflineScanner(instance, new Credentials(user, token), split.getTableId(), authorizations);
+        } else {
+          scanner = conn.createScanner(actualNameForId, authorizations);
+        }
+        if (tableConfig.shouldUseIsolatedScanners()) {
+          log.info("Creating isolated scanner");
+          scanner = new IsolatedScanner(scanner);
+        }
+        if (tableConfig.shouldUseLocalIterators()) {
+          log.info("Using local iterators");
+          scanner = new ClientSideIteratorScanner(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 : tableConfig.getFetchedColumns()) {
+        if (c.getSecond() != null) {
+          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
+          scanner.fetchColumn(c.getFirst(), c.getSecond());
+        } else {
+          log.debug("Fetching column family " + c.getFirst());
+          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);
+      MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME);
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
+      scanner.fetchColumnFamily(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME);
+      scanner.setRange(metadataRange);
+
+      RowIterator rowIter = new RowIterator(scanner);
+
+      KeyExtent lastExtent = null;
+
+      while (rowIter.hasNext()) {
+        Iterator<Map.Entry<Key,Value>> row = rowIter.next();
+        String last = "";
+        KeyExtent extent = null;
+        String location = null;
+
+        while (row.hasNext()) {
+          Map.Entry<Key,Value> entry = row.next();
+          Key key = entry.getKey();
+
+          if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.LastLocationColumnFamily.NAME)) {
+            last = entry.getValue().toString();
+          }
+
+          if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME)
+              || key.getColumnFamily().equals(MetadataSchema.TabletsSection.FutureLocationColumnFamily.NAME)) {
+            location = entry.getValue().toString();
+          }
+
+          if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
+            extent = new KeyExtent(key.getRow(), entry.getValue());
+          }
+
+        }
+
+        if (location != null)
+          return null;
+
+        if (!extent.getTableId().toString().equals(tableId)) {
+          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
+        }
+
+        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
+          throw new AccumuloException(" " + lastExtent + " is not previous extent " + extent);
+        }
+
+        Map<KeyExtent,List<Range>> tabletRanges = binnedRanges.get(last);
+        if (tabletRanges == null) {
+          tabletRanges = new HashMap<KeyExtent,List<Range>>();
+          binnedRanges.put(last, tabletRanges);
+        }
+
+        List<Range> rangeList = tabletRanges.get(extent);
+        if (rangeList == null) {
+          rangeList = new ArrayList<Range>();
+          tabletRanges.put(extent, rangeList);
+        }
+
+        rangeList.add(range);
+
+        if (extent.getEndRow() == null || range.afterEndKey(new Key(extent.getEndRow()).followingKey(PartialKey.ROW))) {
+          break;
+        }
+
+        lastExtent = extent;
+      }
+
+    }
+
+    return binnedRanges;
+  }
+
+  /**
+   * Read the metadata table to get tablets and match up ranges to them.
+   */
+  @Override
+  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+    log.setLevel(getLogLevel(job));
+    validateOptions(job);
+
+    LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
+    Map<String,BatchScanConfig> tableConfigs = getBatchScanConfigs(job);
+    for (Map.Entry<String,BatchScanConfig> tableConfigEntry : tableConfigs.entrySet()) {
+      String tableName = tableConfigEntry.getKey();
+      BatchScanConfig tableConfig = tableConfigEntry.getValue();
+      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, 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), 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, tableName);
+            }
+            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);
+      }
+
+      HashMap<Range,ArrayList<String>> splitsToAdd = null;
+
+      if (!autoAdjust)
+        splitsToAdd = new HashMap<Range,ArrayList<String>>();
+
+      HashMap<String,String> hostNameCache = new HashMap<String,String>();
+      for (Map.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 (Map.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);
+            }
+          }
+        }
+      }
+
+      if (!autoAdjust)
+        for (Map.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/cb97e82a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
new file mode 100644
index 0000000..4e826b9
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapred;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
+import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.util.format.DefaultFormatter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+public class AccumuloMultiTableInputFormat extends AbstractInputFormat {
+
+  /**
+   * Sets the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} 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 setBatchScanConfigs(JobConf job, Map<String,BatchScanConfig> configs) {
+    InputConfigurator.setBatchScanConfigs(CLASS, job, configs);
+  }
+
+  @Override
+  public RecordReader<Key,Value> getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    log.setLevel(getLogLevel(job));
+    InputFormatBase.RecordReaderBase<Key,Value> recordReader = new InputFormatBase.RecordReaderBase<Key,Value>() {
+
+      @Override
+      public boolean next(Key key, Value value) throws IOException {
+        if (scannerIterator.hasNext()) {
+          ++numKeysRead;
+          Map.Entry<Key,Value> entry = scannerIterator.next();
+          key.set(currentKey = entry.getKey());
+          value.set(entry.getValue().get());
+          if (log.isTraceEnabled())
+            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public Key createKey() {
+        return new Key();
+      }
+
+      @Override
+      public Value createValue() {
+        return new Value();
+      }
+
+    };
+    recordReader.initialize(split, job);
+    return recordReader;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cb97e82a/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 0b13966..25c23a6 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
@@ -17,60 +17,25 @@
 package org.apache.accumulo.core.client.mapred;
 
 import java.io.IOException;
-import java.net.InetAddress;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableDeletedException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.impl.OfflineScanner;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.TabletLocator;
-import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
 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.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 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.
@@ -82,179 +47,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * <p>
  * 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.
-   * 
-   * <p>
-   * <b>WARNING:</b> The serialized token is stored in the configuration and shared with all MapReduce tasks. It is BASE64 encoded to provide a charset safe
-   * conversion to a string, and is not intended to be secure.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param principal
-   *          a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param token
-   *          the user's password
-   * @throws AccumuloSecurityException
-   * @since 1.5.0
-   */
-  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.
-   * 
-   * <p>
-   * Stores the password in a file in HDFS and pulls that into the Distributed Cache in an attempt to be more secure than storing it in the Configuration.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param principal
-   *          a valid Accumulo user name (user must have Table.CREATE permission)
-   * @param tokenFile
-   *          the path to the token file
-   * @throws AccumuloSecurityException
-   * @since 1.6.0
-   */
-  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.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return true if the connector has been configured, false otherwise
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static Boolean isConnectorInfoSet(JobConf job) {
-    return InputConfigurator.isConnectorInfoSet(CLASS, job);
-  }
-  
-  /**
-   * Gets the user name from the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the user name
-   * @since 1.5.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   */
-  protected static String getPrincipal(JobConf job) {
-    return InputConfigurator.getPrincipal(CLASS, job);
-  }
-  
-  /**
-   * Gets the serialized token class from either the configuration or the token file.
-   * 
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
-   */
-  @Deprecated
-  protected static String getTokenClass(JobConf job) {
-    return getAuthenticationToken(job).getClass().getName();
-  }
-  
-  /**
-   * Gets the serialized token from either the configuration or the token file.
-   * 
-   * @since 1.5.0
-   * @deprecated since 1.6.0; Use {@link #getAuthenticationToken(JobConf)} instead.
-   */
-  @Deprecated
-  protected static byte[] getToken(JobConf job) {
-    return AuthenticationTokenSerializer.serialize(getAuthenticationToken(job));
-  }
-  
-  /**
-   * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the principal's authentication token
-   * @since 1.6.0
-   * @see #setConnectorInfo(JobConf, String, AuthenticationToken)
-   * @see #setConnectorInfo(JobConf, String, String)
-   */
-  protected static AuthenticationToken getAuthenticationToken(JobConf job) {
-    return InputConfigurator.getAuthenticationToken(CLASS, job);
-  }
-  
-  /**
-   * Configures a {@link ZooKeeperInstance} for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param instanceName
-   *          the Accumulo instance name
-   * @param zooKeepers
-   *          a comma-separated list of zookeeper servers
-   * @since 1.5.0
-   */
-  public static void setZooKeeperInstance(JobConf job, String instanceName, String zooKeepers) {
-    InputConfigurator.setZooKeeperInstance(CLASS, job, instanceName, zooKeepers);
-  }
-  
-  /**
-   * Configures a {@link MockInstance} for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param instanceName
-   *          the Accumulo instance name
-   * @since 1.5.0
-   */
-  public static void setMockInstance(JobConf job, String instanceName) {
-    InputConfigurator.setMockInstance(CLASS, job, instanceName);
-  }
-  
-  /**
-   * Initializes an Accumulo {@link Instance} based on the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return an Accumulo instance
-   * @since 1.5.0
-   * @see #setZooKeeperInstance(JobConf, String, String)
-   * @see #setMockInstance(JobConf, String)
-   */
-  protected static Instance getInstance(JobConf job) {
-    return InputConfigurator.getInstance(CLASS, job);
-  }
-  
-  /**
-   * Sets the log level for this job.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param level
-   *          the logging level
-   * @since 1.5.0
-   */
-  public static void setLogLevel(JobConf job, Level level) {
-    InputConfigurator.setLogLevel(CLASS, job, level);
-  }
-  
-  /**
-   * Gets the log level from this configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the log level
-   * @since 1.5.0
-   * @see #setLogLevel(JobConf, Level)
-   */
-  protected static Level getLogLevel(JobConf job) {
-    return InputConfigurator.getLogLevel(CLASS, job);
-  }
+public abstract class InputFormatBase<K,V> extends AbstractInputFormat<K,V> {
   
   /**
    * Sets the name of the input table, over which this job will scan.
@@ -281,33 +74,6 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
   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.
-   * 
-   * @param job
-   *          the Hadoop job instance to be configured
-   * @param auths
-   *          the user's authorizations
-   * @since 1.5.0
-   */
-  public static void setScanAuthorizations(JobConf job, Authorizations auths) {
-    InputConfigurator.setScanAuthorizations(CLASS, job, auths);
-  }
-  
-  /**
-   * Gets the authorizations to set for the scans from the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return the Accumulo scan authorizations
-   * @since 1.5.0
-   * @see #setScanAuthorizations(JobConf, Authorizations)
-   */
-  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.
    * 
@@ -527,411 +293,26 @@ public abstract class InputFormatBase<K,V> implements InputFormat<K,V> {
     return InputConfigurator.isOfflineScan(CLASS, job);
   }
   
-  /**
-   * Sets the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} 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, Map<String, BatchScanConfig> configs) {
-    checkNotNull(configs);
-    InputConfigurator.setBatchScanConfigs(CLASS, job, configs);
-  }
-  
-  /**
-   * Fetches all {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig}s that have been set on the given Hadoop configuration.
-   * 
-   * <p>
-   * Note this also returns the {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} 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 Map<String,BatchScanConfig> getTableQueryConfigs(JobConf job) {
-    return InputConfigurator.getBatchScanConfigs(CLASS, job);
-  }
-  
-  /**
-   * Fetches a {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} 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 org.apache.accumulo.core.client.mapreduce.BatchScanConfig} for the given table
-   * @since 1.6.0
-   */
-  public static BatchScanConfig getTableQueryConfig(JobConf job, String tableName) {
-    return InputConfigurator.getTableQueryConfig(CLASS, job, tableName);
-  }
-  
-  /**
-   * Initializes an Accumulo {@link TabletLocator} based on the configuration.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @return an Accumulo tablet locator
-   * @throws TableNotFoundException
-   *           if the table name set on the configuration doesn't exist
-   * @since 1.5.0
-   */
-  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 InputFormat}.
-   * 
-   * @param job
-   *          the Hadoop context for the configured job
-   * @throws IOException
-   *           if the context is improperly configured
-   * @since 1.5.0
-   */
-  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.
-   * 
-   * Subclasses must implement {@link #next(Object, Object)} to update key and value, and also to update the following variables:
-   * <ul>
-   * <li>Key {@link #currentKey} (used for progress reporting)</li>
-   * <li>int {@link #numKeysRead} (used for progress reporting)</li>
-   * </ul>
-   */
-  protected abstract static class RecordReaderBase<K,V> implements RecordReader<K,V> {
-    protected long numKeysRead;
-    protected Iterator<Entry<Key,Value>> scannerIterator;
-    protected RangeInputSplit split;
-    
+  protected abstract static class RecordReaderBase<K,V> extends AbstractRecordReader<K,V> {
+
+    @Override
+    protected void setupIterators(JobConf job, Scanner scanner, String tableName) {
+      setupIterators(job, scanner);
+    }
+        
     /**
      * Apply the configured iterators from the configuration to the scanner.
      * 
      * @param job
-     *          the Hadoop context for the configured job
+     *          the Hadoop job configuration
      * @param scanner
      *          the scanner to configure
      */
-    protected void setupIterators(JobConf job, Scanner scanner, String tableName) {
-      BatchScanConfig config = getTableQueryConfig(job, tableName);
-      List<IteratorSetting> iterators = config.getIterators();
-      for (IteratorSetting iterator : iterators)
+    protected void setupIterators(JobConf job, Scanner scanner) {
+      List<IteratorSetting> iterators = getIterators(job);
+      for (IteratorSetting iterator : iterators) 
         scanner.addScanIterator(iterator);
     }
-    
-    /**
-     * Initialize a scanner over the given input split using this task attempt configuration.
-     */
-    public void initialize(InputSplit inSplit, JobConf job) throws IOException {
-      Scanner scanner;
-      split = (RangeInputSplit) inSplit;
-      log.debug("Initializing input split: " + split.getRange());
-      Instance instance = getInstance(job);
-      String user = getPrincipal(job);
-      AuthenticationToken token = getAuthenticationToken(job);
-      Authorizations authorizations = getScanAuthorizations(job);
-      
-      BatchScanConfig 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();
-      if (!(instance instanceof MockInstance)) {
-        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());
-        }
-      }
-      
-      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 (tableConfig.isOfflineScan()) {
-          scanner = new OfflineScanner(instance, new Credentials(user, token), split.getTableId(), authorizations);
-        } else {
-          scanner = conn.createScanner(actualNameForId, authorizations);
-        }
-        if (tableConfig.shouldUseIsolatedScanners()) {
-          log.info("Creating isolated scanner");
-          scanner = new IsolatedScanner(scanner);
-        }
-        if (tableConfig.shouldUseLocalIterators()) {
-          log.info("Using local iterators");
-          scanner = new ClientSideIteratorScanner(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 : tableConfig.getFetchedColumns()) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scanner.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          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);
-      scanner.fetchColumnFamily(TabletsSection.LastLocationColumnFamily.NAME);
-      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.
-   */
-  @Override
-  public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
-    log.setLevel(getLogLevel(job));
-    validateOptions(job);
-    
-    LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
-    Map<String, BatchScanConfig> tableConfigs = getTableQueryConfigs(job);
-    for (Entry<String, BatchScanConfig> tableConfigEntry : tableConfigs.entrySet()) {
-      String tableName = tableConfigEntry.getKey();
-      BatchScanConfig tableConfig = tableConfigEntry.getValue();
-      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, 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), 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, tableName);
-            }
-            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);
-      }
-      
-      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);
-            }
-          }
-        }
-      }
-      
-      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/cb97e82a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index ed29dda..fad414f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.accumulo.core.client.mapreduce;
 
 import java.io.DataInput;
@@ -255,14 +271,14 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
   }
 
   /**
-   * Fetches all {@link BatchScanConfig}s that have been set on the given Hadoop configuration.
+   * Fetches all {@link BatchScanConfig}s that have been set on the given job.
    * 
    * @param context
    *          the Hadoop job instance to be configured
    * @return the {@link BatchScanConfig} objects for the job
    * @since 1.6.0
    */
-  protected static Map<String, BatchScanConfig> getBatchScanConfigs(JobContext context) {
+  protected static Map<String,BatchScanConfig> getBatchScanConfigs(JobContext context) {
     return InputConfigurator.getBatchScanConfigs(CLASS, getConfiguration(context));
   }
 
@@ -332,6 +348,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
 
     /**
      * Configures the iterators on a scanner for the given table name.
+     * 
      * @param context
      *          the Hadoop context for the configured job
      * @param scanner
@@ -548,9 +565,9 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     validateOptions(context);
 
     LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
-    Map<String, BatchScanConfig> tableConfigs = getBatchScanConfigs(context);
-    for (Map.Entry<String, BatchScanConfig> tableConfigEntry : tableConfigs.entrySet()) {
-      
+    Map<String,BatchScanConfig> tableConfigs = getBatchScanConfigs(context);
+    for (Map.Entry<String,BatchScanConfig> tableConfigEntry : tableConfigs.entrySet()) {
+
       String tableName = tableConfigEntry.getKey();
       BatchScanConfig tableConfig = tableConfigEntry.getValue();
 
@@ -567,7 +584,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       TabletLocator tl;
       try {
         if (tableConfig.isOfflineScan()) {
-          binnedRanges = binOfflineTable(context,tableName, ranges);
+          binnedRanges = binOfflineTable(context, 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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cb97e82a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index 4208ff4..de24f68 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.accumulo.core.client.mapreduce;
 
 import org.apache.accumulo.core.client.IteratorSetting;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cb97e82a/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 6e44222..2822c50 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,26 +23,20 @@ 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.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 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;
@@ -57,12 +51,11 @@ import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
 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.
    * 
@@ -71,6 +64,7 @@ public class AccumuloInputFormatTest {
   @Test
   public void testSetIterator() throws IOException {
     JobConf job = new JobConf();
+
     IteratorSetting is = new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator");
     AccumuloInputFormat.addIterator(job, is);
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -78,36 +72,36 @@ public class AccumuloInputFormatTest {
     String iterators = job.get("AccumuloInputFormat.ScanOpts.Iterators");
     assertEquals(new String(Base64.encodeBase64(baos.toByteArray())), iterators);
   }
-  
+
   @Test
   public void testAddIterator() throws IOException {
     JobConf job = new JobConf();
-    
+
     AccumuloInputFormat.addIterator(job, new IteratorSetting(1, "WholeRow", WholeRowIterator.class));
     AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions", "org.apache.accumulo.core.iterators.VersioningIterator"));
     IteratorSetting iter = new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator");
     iter.addOption("v1", "1");
     iter.addOption("junk", "\0omg:!\\xyzzy");
     AccumuloInputFormat.addIterator(job, iter);
-    
+
     List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-    
+
     // Check the list size
     assertTrue(list.size() == 3);
-    
+
     // Walk the list and make sure our settings are correct
     IteratorSetting setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.user.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getName());
     assertEquals(0, setting.getOptions().size());
-    
+
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
     assertEquals("Versions", setting.getName());
     assertEquals(0, setting.getOptions().size());
-    
+
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
@@ -116,7 +110,7 @@ public class AccumuloInputFormatTest {
     assertEquals("1", setting.getOptions().get("v1"));
     assertEquals("\0omg:!\\xyzzy", setting.getOptions().get("junk"));
   }
-  
+
   /**
    * Test adding iterator options where the keys and values contain both the FIELD_SEPARATOR character (':') and ITERATOR_SEPARATOR (',') characters. There
    * should be no exceptions thrown when trying to parse these types of option entries.
@@ -131,12 +125,12 @@ public class AccumuloInputFormatTest {
     someSetting.addOption(key, value);
     JobConf job = new JobConf();
     AccumuloInputFormat.addIterator(job, someSetting);
-    
+
     List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
     assertEquals(1, list.size());
     assertEquals(1, list.get(0).getOptions().size());
     assertEquals(list.get(0).getOptions().get(key), value);
-    
+
     someSetting.addOption(key + "2", value);
     someSetting.setPriority(2);
     someSetting.setName("it2");
@@ -149,7 +143,7 @@ public class AccumuloInputFormatTest {
     assertEquals(list.get(1).getOptions().get(key), value);
     assertEquals(list.get(1).getOptions().get(key + "2"), value);
   }
-  
+
   /**
    * Test getting iterator settings for multiple iterators set
    * 
@@ -158,73 +152,72 @@ public class AccumuloInputFormatTest {
   @Test
   public void testGetIteratorSettings() throws IOException {
     JobConf job = new JobConf();
-    
+
     AccumuloInputFormat.addIterator(job, new IteratorSetting(1, "WholeRow", "org.apache.accumulo.core.iterators.WholeRowIterator"));
     AccumuloInputFormat.addIterator(job, new IteratorSetting(2, "Versions", "org.apache.accumulo.core.iterators.VersioningIterator"));
     AccumuloInputFormat.addIterator(job, new IteratorSetting(3, "Count", "org.apache.accumulo.core.iterators.CountingIterator"));
-    
+
     List<IteratorSetting> list = AccumuloInputFormat.getIterators(job);
-    
+
     // Check the list size
     assertTrue(list.size() == 3);
-    
+
     // Walk the list and make sure our settings are correct
     IteratorSetting setting = list.get(0);
     assertEquals(1, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.WholeRowIterator", setting.getIteratorClass());
     assertEquals("WholeRow", setting.getName());
-    
+
     setting = list.get(1);
     assertEquals(2, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.VersioningIterator", setting.getIteratorClass());
     assertEquals("Versions", setting.getName());
-    
+
     setting = list.get(2);
     assertEquals(3, setting.getPriority());
     assertEquals("org.apache.accumulo.core.iterators.CountingIterator", setting.getIteratorClass());
     assertEquals("Count", setting.getName());
-    
+
   }
-  
+
   @Test
   public void testSetRegex() throws IOException {
     JobConf job = new JobConf();
-    
+
     String regex = ">\"*%<>\'\\";
-    
+
     IteratorSetting is = new IteratorSetting(50, regex, RegExFilter.class);
     RegExFilter.setRegexs(is, regex, null, null, null, false);
     AccumuloInputFormat.addIterator(job, is);
-    
+
     assertTrue(regex.equals(AccumuloInputFormat.getIterators(job).get(0).getName()));
   }
-  
+
   private static AssertionError e1 = null;
   private static AssertionError e2 = null;
-  
+
   private static class MRTester extends Configured implements Tool {
     private static class TestMapper implements Mapper<Key,Value,Key,Value> {
       Key key = null;
       int count = 0;
-      
+
       @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(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
-          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
         } catch (AssertionError e) {
           e1 = e;
         }
         key = new Key(k);
         count++;
       }
-      
+
       @Override
       public void configure(JobConf job) {}
-      
+
       @Override
       public void close() throws IOException {
         try {
@@ -233,124 +226,59 @@ public class AccumuloInputFormatTest {
           e2 = e;
         }
       }
-      
+
     }
-    
+
     @Override
     public int run(String[] args) throws Exception {
-      
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
+
+      if (args.length != 3) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
       }
-      
+
       String user = args[0];
       String pass = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-      
+      String table = args[2];
+
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
-      
+
       job.setInputFormat(AccumuloInputFormat.class);
-      
+
       AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+      AccumuloInputFormat.setInputTableName(job, table);
       AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-      
-      BatchScanConfig tableConfig1 = new BatchScanConfig();
-      BatchScanConfig tableConfig2 = new BatchScanConfig();
-
-      Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
-      configMap.put(table1, tableConfig1);
-      configMap.put(table2, tableConfig2);
-      
-      AccumuloInputFormat.setTableQueryConfigs(job, configMap);
-      
+
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
       job.setMapOutputValueClass(Value.class);
       job.setOutputFormat(NullOutputFormat.class);
-      
+
       job.setNumReduceTasks(0);
-      
+
       return JobClient.runJob(job).isSuccessful() ? 0 : 1;
     }
-    
+
     public static void main(String[] args) throws Exception {
       assertEquals(0, ToolRunner.run(CachedConfiguration.getInstance(), new MRTester(), args));
     }
   }
-  
+
   @Test
   public void testMap() throws Exception {
     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 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);
+      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);
     }
     bw.close();
-    bw2.close();
-    
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
+
+    MRTester.main(new String[] {"root", "", TEST_TABLE_1});
     assertNull(e1);
     assertNull(e2);
   }
-  
-  /**
-   * Verify {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects get correctly serialized in the JobContext.
-   */
-  @Test
-  public void testTableQueryConfigSerialization() throws IOException {
-    
-    JobConf job = new JobConf();
-    
-    BatchScanConfig table1 = new BatchScanConfig().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")));
-    
-    BatchScanConfig table2 = new BatchScanConfig().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")));
-    
-    Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
-    configMap.put(TEST_TABLE_1, table1);
-    configMap.put(TEST_TABLE_2, table2);
-    AccumuloInputFormat.setTableQueryConfigs(job, configMap);
-
-    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();
-    
-    BatchScanConfig tableConfig = new BatchScanConfig().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")));
-    
-    Map<String, BatchScanConfig> configMap = new HashMap<String, BatchScanConfig>();
-    configMap.put(TEST_TABLE_1, tableConfig);
-    
-    AccumuloInputFormat.setTableQueryConfigs(job, configMap);
-    AccumuloInputFormat.setInputTableName(job, TEST_TABLE_2);
-    AccumuloInputFormat.setRanges(job, tableConfig.getRanges());
-    AccumuloInputFormat.fetchColumns(job, tableConfig.getFetchedColumns());
-    AccumuloInputFormat.addIterator(job, tableConfig.getIterators().get(0));
-    
-    assertEquals(tableConfig, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_1));
-    assertEquals(tableConfig, AccumuloInputFormat.getTableQueryConfig(job, TEST_TABLE_2));
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cb97e82a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
new file mode 100644
index 0000000..97f8d72
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.mapreduce.BatchScanConfig;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+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.util.CachedConfiguration;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+public class AccumuloMultiTableInputFormatTest {
+
+  private static final String PREFIX = AccumuloMultiTableInputFormatTest.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";
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
+      Key key = null;
+      int count = 0;
+
+      @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(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;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(100, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 4) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
+      }
+
+      String user = args[0];
+      String pass = args[1];
+      String table1 = args[2];
+      String table2 = args[3];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+      AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
+
+      BatchScanConfig tableConfig1 = new BatchScanConfig();
+      BatchScanConfig tableConfig2 = new BatchScanConfig();
+
+      Map<String,BatchScanConfig> configMap = new HashMap<String,BatchScanConfig>();
+      configMap.put(table1, tableConfig1);
+      configMap.put(table2, tableConfig2);
+
+      AccumuloMultiTableInputFormat.setBatchScanConfigs(job, configMap);
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      assertEquals(0, ToolRunner.run(CachedConfiguration.getInstance(), new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void testMap() throws Exception {
+    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 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, TEST_TABLE_2});
+    assertNull(e1);
+    assertNull(e2);
+  }
+
+  /**
+   * Verify {@link org.apache.accumulo.core.client.mapreduce.BatchScanConfig} objects get correctly serialized in the JobContext.
+   */
+  @Test
+  public void testTableQueryConfigSerialization() throws IOException {
+
+    JobConf job = new JobConf();
+
+    BatchScanConfig table1 = new BatchScanConfig().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")));
+
+    BatchScanConfig table2 = new BatchScanConfig().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")));
+
+    Map<String,BatchScanConfig> configMap = new HashMap<String,BatchScanConfig>();
+    configMap.put(TEST_TABLE_1, table1);
+    configMap.put(TEST_TABLE_2, table2);
+    AccumuloMultiTableInputFormat.setBatchScanConfigs(job, configMap);
+
+    assertEquals(table1, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_1));
+    assertEquals(table2, AccumuloMultiTableInputFormat.getBatchScanConfig(job, TEST_TABLE_2));
+  }
+}