You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2013/11/24 00:52:10 UTC

[22/30] Merge branch '1.5.1-SNAPSHOT' into 1.6.1-SNAPSHOT

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 5c2777d,0000000..2b7e958
mode 100644,000000..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,741 -1,0 +1,679 @@@
 +/*
 + * 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;
- import java.io.DataOutput;
 +import java.io.IOException;
 +import java.lang.reflect.Method;
- import java.math.BigInteger;
 +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 org.apache.accumulo.core.client.AccumuloException;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +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.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.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Tables;
 +import org.apache.accumulo.core.client.impl.TabletLocator;
 +import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
- import org.apache.accumulo.core.data.ByteSequence;
 +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.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.conf.Configuration;
 +import org.apache.hadoop.io.Text;
- import org.apache.hadoop.io.Writable;
 +import org.apache.hadoop.mapreduce.InputFormat;
 +import org.apache.hadoop.mapreduce.InputSplit;
 +import org.apache.hadoop.mapreduce.Job;
 +import org.apache.hadoop.mapreduce.JobContext;
 +import org.apache.hadoop.mapreduce.RecordReader;
 +import org.apache.hadoop.mapreduce.TaskAttemptContext;
 +import org.apache.log4j.Level;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * An abstract input format to provide shared methods common to all other input format classes. At the very least, any classes inheriting from this class will
 + * need to define their own {@link RecordReader}.
 + */
 +public abstract class AbstractInputFormat<K,V> extends 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(Job job, String principal, AuthenticationToken token) throws AccumuloSecurityException {
 +    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, token);
 +  }
 +
 +  /**
 +   * Sets the connector information needed to communicate with Accumulo in this job.
 +   * 
 +   * <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(Job job, String principal, String tokenFile) throws AccumuloSecurityException {
 +    InputConfigurator.setConnectorInfo(CLASS, job.getConfiguration(), principal, tokenFile);
 +  }
 +
 +  /**
 +   * Determines if the connector has been configured.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return true if the connector has been configured, false otherwise
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   */
 +  protected static Boolean isConnectorInfoSet(JobContext context) {
 +    return InputConfigurator.isConnectorInfoSet(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * Gets the user name from the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the user name
 +   * @since 1.5.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   */
 +  protected static String getPrincipal(JobContext context) {
 +    return InputConfigurator.getPrincipal(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * 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(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static String getTokenClass(JobContext context) {
 +    return getAuthenticationToken(context).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(JobContext)} instead.
 +   */
 +  @Deprecated
 +  protected static byte[] getToken(JobContext context) {
 +    return AuthenticationToken.AuthenticationTokenSerializer.serialize(getAuthenticationToken(context));
 +  }
 +
 +  /**
 +   * Gets the authenticated token from either the specified token file or directly from the configuration, whichever was used when the job was configured.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the principal's authentication token
 +   * @since 1.6.0
 +   * @see #setConnectorInfo(Job, String, AuthenticationToken)
 +   * @see #setConnectorInfo(Job, String, String)
 +   */
 +  protected static AuthenticationToken getAuthenticationToken(JobContext context) {
 +    return InputConfigurator.getAuthenticationToken(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * 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
 +   * @deprecated since 1.6.0; Use {@link #setZooKeeperInstance(Job, ClientConfiguration)} instead.
 +   */
 +  @Deprecated
 +  public static void setZooKeeperInstance(Job job, String instanceName, String zooKeepers) {
 +    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), instanceName, zooKeepers);
 +  }
 +
 +  /**
 +   * Configures a {@link org.apache.accumulo.core.client.ZooKeeperInstance} for this job.
 +   *
 +   * @param job
 +   *          the Hadoop job instance to be configured
 +   * @param clientConfig
 +   *          client configuration containing connection options
 +   * @since 1.6.0
 +   */
 +  public static void setZooKeeperInstance(Job job, ClientConfiguration clientConfig) {
 +    InputConfigurator.setZooKeeperInstance(CLASS, job.getConfiguration(), clientConfig);
 +  }
 +
 +  /**
 +   * 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(Job job, String instanceName) {
 +    InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
 +  }
 +
 +  /**
 +   * Initializes an Accumulo {@link org.apache.accumulo.core.client.Instance} based on the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return an Accumulo instance
 +   * @since 1.5.0
 +   * @see #setZooKeeperInstance(Job, String, String)
 +   * @see #setMockInstance(Job, String)
 +   */
 +  protected static Instance getInstance(JobContext context) {
 +    return InputConfigurator.getInstance(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * 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(Job job, Level level) {
 +    InputConfigurator.setLogLevel(CLASS, job.getConfiguration(), level);
 +  }
 +
 +  /**
 +   * Gets the log level from this configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the log level
 +   * @since 1.5.0
 +   * @see #setLogLevel(Job, Level)
 +   */
 +  protected static Level getLogLevel(JobContext context) {
 +    return InputConfigurator.getLogLevel(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * 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
 +   */
 +  public static void setScanAuthorizations(Job job, Authorizations auths) {
 +    InputConfigurator.setScanAuthorizations(CLASS, job.getConfiguration(), auths);
 +  }
 +
 +  /**
 +   * Gets the authorizations to set for the scans from the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @return the Accumulo scan authorizations
 +   * @since 1.5.0
 +   * @see #setScanAuthorizations(Job, Authorizations)
 +   */
 +  protected static Authorizations getScanAuthorizations(JobContext context) {
 +    return InputConfigurator.getScanAuthorizations(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * Fetches all {@link InputTableConfig}s that have been set on the given job.
 +   * 
 +   * @param context
 +   *          the Hadoop job instance to be configured
 +   * @return the {@link InputTableConfig} objects for the job
 +   * @since 1.6.0
 +   */
 +  protected static Map<String,InputTableConfig> getInputTableConfigs(JobContext context) {
 +    return InputConfigurator.getInputTableConfigs(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * Fetches a {@link InputTableConfig} 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 context
 +   *          the Hadoop job instance to be configured
 +   * @param tableName
 +   *          the table name for which to grab the config object
 +   * @return the {@link InputTableConfig} for the given table
 +   * @since 1.6.0
 +   */
 +  protected static InputTableConfig getInputTableConfig(JobContext context, String tableName) {
 +    return InputConfigurator.getInputTableConfig(CLASS, getConfiguration(context), tableName);
 +  }
 +
 +  /**
 +   * Initializes an Accumulo {@link org.apache.accumulo.core.client.impl.TabletLocator} based on the configuration.
 +   * 
 +   * @param context
 +   *          the Hadoop context for the configured job
 +   * @param table
 +   *          the table for which to initialize the locator
 +   * @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.6.0
 +   */
 +  protected static TabletLocator getTabletLocator(JobContext context, String table) throws TableNotFoundException {
 +    return InputConfigurator.getTabletLocator(CLASS, getConfiguration(context), table);
 +  }
 +
 +  // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
 +  /**
 +   * Check whether a configuration is fully configured to be used with an Accumulo {@link org.apache.hadoop.mapreduce.InputFormat}.
 +   * 
 +   * @param context
 +   *          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(JobContext context) throws IOException {
 +    InputConfigurator.validateOptions(CLASS, getConfiguration(context));
 +  }
 +
 +  /**
 +   * An abstract base class to be used to create {@link org.apache.hadoop.mapreduce.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 #nextKeyValue()} and use it to update the following variables:
 +   * <ul>
 +   * <li>K {@link #currentK}</li>
 +   * <li>V {@link #currentV}</li>
 +   * <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> extends 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 context
 +     *          the Hadoop context for the configured job
 +     * @param scanner
 +     *          the scanner for which to configure the iterators
 +     * @param tableName
 +     *          the table name for which the scanner is configured
 +     * @since 1.6.0
 +     */
-     protected abstract void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName);
++    protected abstract void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName, RangeInputSplit split);
 +
 +    /**
 +     * Initialize a scanner over the given input split using this task attempt configuration.
 +     */
 +    @Override
 +    public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
 +
 +      Scanner scanner;
 +      split = (RangeInputSplit) inSplit;
 +      log.debug("Initializing input split: " + split.getRange());
-       Instance instance = getInstance(attempt);
-       String principal = getPrincipal(attempt);
-       AuthenticationToken token = getAuthenticationToken(attempt);
-       Authorizations authorizations = getScanAuthorizations(attempt);
++      
++      Instance instance = split.getInstance();
++      if (null == instance) {
++        instance = getInstance(attempt);
++      }
++
++      String principal = split.getPrincipal();
++      if (null == principal) {
++        principal = getPrincipal(attempt);
++      }
++
++      AuthenticationToken token = split.getToken();
++      if (null == token) {
++        token = getAuthenticationToken(attempt);
++      }
++
++      Authorizations authorizations = split.getAuths();
++      if (null == authorizations) {
++        authorizations = getScanAuthorizations(attempt);
++      }
++
++      String table = split.getTableName();
 +
 +      // in case the table name changed, we can still use the previous name for terms of configuration,
 +      // but the scanner will use the table id resolved at job setup time
 +      InputTableConfig tableConfig = getInputTableConfig(attempt, split.getTableName());
++      
++      Boolean isOffline = split.isOffline();
++      if (null == isOffline) {
++        isOffline = tableConfig.isOfflineScan();
++      }
++
++      Boolean isIsolated = split.isIsolatedScan();
++      if (null == isIsolated) {
++        isIsolated = tableConfig.shouldUseIsolatedScanners();
++      }
++
++      Boolean usesLocalIterators = split.usesLocalIterators();
++      if (null == usesLocalIterators) {
++        usesLocalIterators = tableConfig.shouldUseLocalIterators();
++      }
++      
++      List<IteratorSetting> iterators = split.getIterators();
++      if (null == iterators) {
++        iterators = tableConfig.getIterators();
++      }
++      
++      Collection<Pair<Text,Text>> columns = split.getFetchedColumns();
++      if (null == columns) {
++        columns = tableConfig.getFetchedColumns();
++      }
 +
 +      try {
 +        log.debug("Creating connector with user: " + principal);
-         log.debug("Creating scanner for table: " + split.getTableName());
++        log.debug("Creating scanner for table: " + table);
 +        log.debug("Authorizations are: " + authorizations);
-         if (tableConfig.isOfflineScan()) {
++        if (isOffline) {
 +          scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
 +        } else if (instance instanceof MockInstance) {
 +          scanner = instance.getConnector(principal, token).createScanner(split.getTableName(), authorizations);
 +        } else {
 +          scanner = new ScannerImpl(instance, new Credentials(principal, token), split.getTableId(), authorizations);
 +        }
-         if (tableConfig.shouldUseIsolatedScanners()) {
++        if (isIsolated) {
 +          log.info("Creating isolated scanner");
 +          scanner = new IsolatedScanner(scanner);
 +        }
-         if (tableConfig.shouldUseLocalIterators()) {
++        if (usesLocalIterators) {
 +          log.info("Using local iterators");
 +          scanner = new ClientSideIteratorScanner(scanner);
 +        }
-         setupIterators(attempt, scanner, split.getTableName());
++        
++        setupIterators(attempt, scanner, split.getTableName(), split);
 +      } catch (Exception e) {
 +        throw new IOException(e);
 +      }
 +
 +      // setup a scanner within the bounds of this split
-       for (Pair<Text,Text> c : tableConfig.getFetchedColumns()) {
++      for (Pair<Text,Text> c : columns) {
 +        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 float getProgress() throws IOException {
 +      if (numKeysRead > 0 && currentKey == null)
 +        return 1.0f;
 +      return split.getProgress(currentKey);
 +    }
 +
 +    protected K currentK = null;
 +    protected V currentV = null;
 +    protected Key currentKey = null;
 +    protected Value currentValue = null;
 +
 +    @Override
 +    public K getCurrentKey() throws IOException, InterruptedException {
 +      return currentK;
 +    }
 +
 +    @Override
 +    public V getCurrentValue() throws IOException, InterruptedException {
 +      return currentV;
 +    }
 +  }
 +
 +  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, String tableId, List<Range> ranges) throws TableNotFoundException,
 +      AccumuloException, AccumuloSecurityException {
 +
 +    Instance instance = getInstance(context);
 +    Connector conn = instance.getConnector(getPrincipal(context), getAuthenticationToken(context));
 +
 +    return InputConfigurator.binOffline(tableId, ranges, instance, conn);
 +  }
 +
 +  /**
 +   * Gets the splits of the tables that have been set on the job.
 +   * 
 +   * @param context
 +   *          the configuration of the job
 +   * @return the splits from the tables based on the ranges.
 +   * @throws java.io.IOException
 +   *           if a table set on the job doesn't exist or an error occurs initializing the tablet locator
 +   */
 +  public List<InputSplit> getSplits(JobContext context) throws IOException {
-     log.setLevel(getLogLevel(context));
++    Level logLevel = getLogLevel(context);
++    log.setLevel(logLevel);
 +    validateOptions(context);
 +
 +    LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
 +    Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(context);
 +    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
 +
 +      String tableName = tableConfigEntry.getKey();
 +      InputTableConfig tableConfig = tableConfigEntry.getValue();
++      
++      Instance instance = getInstance(context);
++      boolean mockInstance;
++      String tableId;
++      // resolve table name to id once, and use id from this point forward
++      if (instance instanceof MockInstance) {
++        tableId = "";
++        mockInstance = true;
++      } else {
++        try {
++          tableId = Tables.getTableId(instance, tableName);
++        } catch (TableNotFoundException e) {
++          throw new IOException(e);
++        }
++        mockInstance = false;
++      }
++      
++      Authorizations auths = getScanAuthorizations(context);
++      String principal = getPrincipal(context);
++      AuthenticationToken token = getAuthenticationToken(context);
 +
 +      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 {
-         // resolve table name to id once, and use id from this point forward
-         Instance instance = getInstance(context);
-         if (instance instanceof MockInstance)
-           tableId = "";
-         else
-           tableId = Tables.getTableId(instance, tableName);
 +        if (tableConfig.isOfflineScan()) {
 +          binnedRanges = binOfflineTable(context, tableId, 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(context, tableId, ranges);
 +
 +          }
 +        } else {
 +          tl = getTabletLocator(context, tableId);
 +          // 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(context), getAuthenticationToken(context));
 +
 +          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);
 +            }
 +            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}));
++              RangeInputSplit split = new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location});
++              
++              split.setOffline(tableConfig.isOfflineScan());
++              split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
++              split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
++              split.setMockInstance(mockInstance);
++              split.setFetchedColumns(tableConfig.getFetchedColumns());
++              split.setPrincipal(principal);
++              split.setToken(token);
++              split.setInstanceName(instance.getInstanceName());
++              split.setZooKeepers(instance.getZooKeepers());
++              split.setAuths(auths);
++              split.setIterators(tableConfig.getIterators());
++              split.setLogLevel(logLevel);
++              
++              splits.add(split);
 +            } 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;
-   }
- 
-   /**
-    * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
-    */
-   public static class RangeInputSplit extends InputSplit implements Writable {
-     private Range range;
-     private String[] locations;
-     private String tableId;
-     private String tableName;
- 
-     public RangeInputSplit() {
-       range = new Range();
-       locations = new String[0];
-       tableId = "";
-       tableName = "";
-     }
- 
-     public RangeInputSplit(RangeInputSplit split) throws IOException {
-       this.setRange(split.getRange());
-       this.setLocations(split.getLocations());
-       this.setTableName(split.getTableName());
-       this.setTableId(split.getTableId());
-     }
- 
-     protected RangeInputSplit(String table, String tableId, Range range, String[] locations) {
-       this.range = range;
-       this.locations = locations;
-       this.tableName = table;
-       this.tableId = tableId;
-     }
- 
-     public Range getRange() {
-       return range;
-     }
- 
-     public void setRange(Range range) {
-       this.range = range;
-     }
- 
-     public String getTableName() {
-       return tableName;
-     }
- 
-     public void setTableName(String tableName) {
-       this.tableName = tableName;
-     }
- 
-     public void setTableId(String tableId) {
-       this.tableId = tableId;
-     }
- 
-     public String getTableId() {
-       return tableId;
-     }
- 
-     private static byte[] extractBytes(ByteSequence seq, int numBytes) {
-       byte[] bytes = new byte[numBytes + 1];
-       bytes[0] = 0;
-       for (int i = 0; i < numBytes; i++) {
-         if (i >= seq.length())
-           bytes[i + 1] = 0;
-         else
-           bytes[i + 1] = seq.byteAt(i);
-       }
-       return bytes;
-     }
- 
-     public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
-       int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
-       BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
-       BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
-       BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
-       return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
-     }
- 
-     public float getProgress(Key currentKey) {
-       if (currentKey == null)
-         return 0f;
-       if (range.getStartKey() != null && range.getEndKey() != null) {
-         if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW)) {
-           // just look at the row progress
-           return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
-         } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM)) {
-           // just look at the column family progress
-           return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
-         } else if (!range.getStartKey().equals(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL)) {
-           // just look at the column qualifier progress
-           return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
++        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
++          RangeInputSplit split = new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0]));
++
++          split.setOffline(tableConfig.isOfflineScan());
++          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
++          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
++          split.setMockInstance(mockInstance);
++          split.setFetchedColumns(tableConfig.getFetchedColumns());
++          split.setPrincipal(principal);
++          split.setToken(token);
++          split.setInstanceName(instance.getInstanceName());
++          split.setZooKeepers(instance.getZooKeepers());
++          split.setAuths(auths);
++          split.setIterators(tableConfig.getIterators());
++          split.setLogLevel(logLevel);
++          
++          splits.add(split);
 +        }
-       }
-       // if we can't figure it out, then claim no progress
-       return 0f;
-     }
- 
-     /**
-      * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
-      */
-     @Override
-     public long getLength() throws IOException {
-       Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
-       Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
-       int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
-       long diff = 0;
- 
-       byte[] start = startRow.getBytes();
-       byte[] stop = stopRow.getBytes();
-       for (int i = 0; i < maxCommon; ++i) {
-         diff |= 0xff & (start[i] ^ stop[i]);
-         diff <<= Byte.SIZE;
-       }
- 
-       if (startRow.getLength() != stopRow.getLength())
-         diff |= 0xff;
- 
-       return diff + 1;
-     }
- 
-     @Override
-     public String[] getLocations() throws IOException {
-       return locations;
-     }
- 
-     public void setLocations(String[] locations) {
-       this.locations = locations;
-     }
- 
-     @Override
-     public void readFields(DataInput in) throws IOException {
-       range.readFields(in);
-       tableName = in.readUTF();
-       tableId = in.readUTF();
-       int numLocs = in.readInt();
-       locations = new String[numLocs];
-       for (int i = 0; i < numLocs; ++i)
-         locations[i] = in.readUTF();
-     }
- 
-     @Override
-     public void write(DataOutput out) throws IOException {
-       range.write(out);
-       out.writeUTF(tableName);
-       out.writeUTF(tableId);
-       out.writeInt(locations.length);
-       for (int i = 0; i < locations.length; ++i)
-         out.writeUTF(locations[i]);
 +    }
++    return splits;
 +  }
 +
 +  // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
 +  static Configuration getConfiguration(JobContext context) {
 +    try {
 +      Class<?> c = AbstractInputFormat.class.getClassLoader().loadClass("org.apache.hadoop.mapreduce.JobContext");
 +      Method m = c.getMethod("getConfiguration");
 +      Object o = m.invoke(context, new Object[0]);
 +      return (Configuration) o;
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 9a339be,0220339..a1340c3
--- 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
@@@ -50,8 -50,17 +51,18 @@@ public class AccumuloInputFormat extend
    @Override
    public RecordReader<Key,Value> createRecordReader(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
      log.setLevel(getLogLevel(context));
+     
+     // Override the log level from the configuration as if the RangeInputSplit has one it's the more correct one to use.
+     if (split instanceof RangeInputSplit) {
+       RangeInputSplit risplit = (RangeInputSplit) split;
+       Level level = risplit.getLogLevel();
+       if (null != level) {
+         log.setLevel(level);
+       }
+     }
+ 
      return new RecordReaderBase<Key,Value>() {
 +
        @Override
        public boolean nextKeyValue() throws IOException, InterruptedException {
          if (scannerIterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index 357bf38,0000000..9e86c7b
mode 100644,000000..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,97 -1,0 +1,101 @@@
 +/*
 + * 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 static com.google.common.base.Preconditions.checkNotNull;
 +
 +import java.io.IOException;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.client.ClientConfiguration;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.util.format.DefaultFormatter;
 +import org.apache.hadoop.mapreduce.InputSplit;
 +import org.apache.hadoop.mapreduce.Job;
 +import org.apache.hadoop.mapreduce.RecordReader;
 +import org.apache.hadoop.mapreduce.TaskAttemptContext;
 +
 +/**
 + * This class allows MapReduce jobs to use multiple Accumulo tables as the source of data. This {@link org.apache.hadoop.mapreduce.InputFormat} provides keys
 + * and values of type {@link Key} and {@link Value} to the Map function.
 + * 
 + * The user must specify the following via static configurator methods:
 + * 
 + * <ul>
 + * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
 + * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(Job, Authorizations)}
 + * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
 + * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(Job, Map)}
 + * </ul>
 + * 
 + * Other static methods are optional.
 + */
 +public class AccumuloMultiTableInputFormat extends AbstractInputFormat<Key,Value> {
 +
 +  /**
 +   * Sets the {@link InputTableConfig} 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 setInputTableConfigs(Job job, Map<String,InputTableConfig> configs) {
 +    checkNotNull(configs);
 +    InputConfigurator.setInputTableConfigs(CLASS, getConfiguration(job), configs);
 +  }
 +
 +  @Override
 +  public RecordReader<Key,Value> createRecordReader(InputSplit inputSplit, TaskAttemptContext context) throws IOException, InterruptedException {
 +    log.setLevel(getLogLevel(context));
 +    return new AbstractRecordReader<Key,Value>() {
 +      @Override
 +      public boolean nextKeyValue() throws IOException, InterruptedException {
 +        if (scannerIterator.hasNext()) {
 +          ++numKeysRead;
 +          Map.Entry<Key,Value> entry = scannerIterator.next();
 +          currentK = currentKey = entry.getKey();
 +          currentV = currentValue = entry.getValue();
 +          if (log.isTraceEnabled())
 +            log.trace("Processing key/value pair: " + DefaultFormatter.formatEntry(entry, true));
 +          return true;
 +        }
 +        return false;
 +      }
 +
 +      @Override
-       protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
-         List<IteratorSetting> iterators = getInputTableConfig(context, tableName).getIterators();
++      protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName, RangeInputSplit split) {
++        List<IteratorSetting> iterators = split.getIterators(); 
++        if (null == iterators) {
++          iterators = getInputTableConfig(context, tableName).getIterators();
++        }
++        
 +        for (IteratorSetting setting : iterators) {
 +          scanner.addScanIterator(setting);
 +        }
 +      }
 +    };
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/aec43807/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
index b85bfd6,5e246c4..9525796
--- 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
@@@ -320,25 -556,769 +320,28 @@@ public abstract class InputFormatBase<K
       *          the Hadoop context for the configured job
       * @param scanner
       *          the scanner to configure
 +     * @since 1.6.0
       */
 -    protected void setupIterators(List<IteratorSetting> iterators, Scanner scanner) {
 -      for (IteratorSetting iterator : iterators) {
 -        scanner.addScanIterator(iterator);
 -      }
 +    @Override
-     protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName) {
-       setupIterators(context, scanner);
++    protected void setupIterators(TaskAttemptContext context, Scanner scanner, String tableName, RangeInputSplit split) {
++      setupIterators(context, scanner, split);
      }
- 
+     
      /**
 -     * Initialize a scanner over the given input split using this task attempt configuration.
 +     * Apply the configured iterators from the configuration to the scanner.
 +     * 
 +     * @param context
 +     *          the Hadoop context for the configured job
 +     * @param scanner
 +     *          the scanner to configure
       */
-     protected void setupIterators(TaskAttemptContext context, Scanner scanner) {
-       List<IteratorSetting> iterators = getIterators(context);
 -    @Override
 -    public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IOException {
 -      Scanner scanner;
 -      split = (RangeInputSplit) inSplit;
 -      log.debug("Initializing input split: " + split.getRange());
 -
 -      Instance instance = split.getInstance();
 -      if (null == instance) {
 -        instance = getInstance(attempt);
 -      }
 -
 -      String principal = split.getPrincipal();
 -      if (null == principal) {
 -        principal = getPrincipal(attempt);
 -      }
 -
 -      AuthenticationToken token = split.getToken();
 -      if (null == token) {
 -        String tokenClass = getTokenClass(attempt);
 -        byte[] tokenBytes = getToken(attempt);
 -        try {
 -          token = CredentialHelper.extractToken(tokenClass, tokenBytes);
 -        } catch (AccumuloSecurityException e) {
 -          throw new IOException(e);
 -        }
 -      }
 -
 -      Authorizations authorizations = split.getAuths();
 -      if (null == authorizations) {
 -        authorizations = getScanAuthorizations(attempt);
 -      }
 -
 -      String table = split.getTable();
 -      if (null == table) {
 -        table = getInputTableName(attempt);
 -      }
 -      
 -      Boolean isOffline = split.isOffline();
 -      if (null == isOffline) {
 -        isOffline = isOfflineScan(attempt);
 -      }
 -
 -      Boolean isIsolated = split.isIsolatedScan();
 -      if (null == isIsolated) {
 -        isIsolated = isIsolated(attempt);
 -      }
 -
 -      Boolean usesLocalIterators = split.usesLocalIterators();
 -      if (null == usesLocalIterators) {
 -        usesLocalIterators = usesLocalIterators(attempt);
 -      }
 -      
 -      List<IteratorSetting> iterators = split.getIterators();
++    protected void setupIterators(TaskAttemptContext context, Scanner scanner, RangeInputSplit split) {
++      List<IteratorSetting> iterators = split.getIterators(); 
+       if (null == iterators) {
 -        iterators = getIterators(attempt);
 -      }
 -      
 -      Set<Pair<Text,Text>> columns = split.getFetchedColumns();
 -      if (null == columns) {
 -        columns = getFetchedColumns(attempt);
 -      }
 -      
 -      try {
 -        log.debug("Creating connector with user: " + principal);
 -        Connector conn = instance.getConnector(principal, token);
 -        log.debug("Creating scanner for table: " + table);
 -        log.debug("Authorizations are: " + authorizations);
 -        if (isOfflineScan(attempt)) {
 -          String tokenClass = token.getClass().getCanonicalName();
 -          ByteBuffer tokenBuffer = ByteBuffer.wrap(CredentialHelper.toBytes(token));
 -          scanner = new OfflineScanner(instance, new TCredentials(principal, tokenClass, tokenBuffer, instance.getInstanceID()), Tables.getTableId(
 -              instance, table), authorizations);
 -        } else {
 -          scanner = conn.createScanner(table, authorizations);
 -        }
 -        if (isIsolated) {
 -          log.info("Creating isolated scanner");
 -          scanner = new IsolatedScanner(scanner);
 -        }
 -        if (usesLocalIterators) {
 -          log.info("Using local iterators");
 -          scanner = new ClientSideIteratorScanner(scanner);
 -        }
 -        setupIterators(iterators, scanner);
 -      } catch (Exception e) {
 -        throw new IOException(e);
 -      }
 -      
 -      // setup a scanner within the bounds of this split
 -      for (Pair<Text,Text> c : columns) {
 -        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 float getProgress() throws IOException {
 -      if (numKeysRead > 0 && currentKey == null)
 -        return 1.0f;
 -      return split.getProgress(currentKey);
 -    }
 -    
 -    protected K currentK = null;
 -    protected V currentV = null;
 -    protected Key currentKey = null;
 -    protected Value currentValue = null;
 -    
 -    @Override
 -    public K getCurrentKey() throws IOException, InterruptedException {
 -      return currentK;
 -    }
 -    
 -    @Override
 -    public V getCurrentValue() throws IOException, InterruptedException {
 -      return currentV;
 -    }
 -  }
 -  
 -  Map<String,Map<KeyExtent,List<Range>>> binOfflineTable(JobContext context, String tableName, List<Range> ranges) throws TableNotFoundException,
 -      AccumuloException, AccumuloSecurityException {
 -    
 -    Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
 -    
 -    Instance instance = getInstance(context);
 -    Connector conn = instance.getConnector(getPrincipal(context), CredentialHelper.extractToken(getTokenClass(context), getToken(context)));
 -    String tableId = Tables.getTableId(instance, tableName);
 -    
 -    if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 -      Tables.clearCache(instance);
 -      if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) {
 -        throw new AccumuloException("Table is online " + tableName + "(" + tableId + ") cannot scan table in offline mode ");
++        iterators = getIterators(context);
+       }
 -    }
 -    
 -    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(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 -      Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
 -      scanner.fetchColumnFamily(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY);
 -      scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
 -      scanner.fetchColumnFamily(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY);
 -      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(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
 -            last = entry.getValue().toString();
 -          }
 -          
 -          if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)
 -              || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) {
 -            location = entry.getValue().toString();
 -          }
 -          
 -          if (Constants.METADATA_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 List<InputSplit> getSplits(JobContext context) throws IOException {
 -    Level logLevel = getLogLevel(context);
 -    log.setLevel(logLevel);
 -    
 -    validateOptions(context);
 -    
 -    String tableName = getInputTableName(context);
 -    boolean autoAdjust = getAutoAdjustRanges(context);
 -    List<Range> ranges = autoAdjust ? Range.mergeOverlapping(getRanges(context)) : getRanges(context);
 -    Instance instance = getInstance(context);
 -    boolean offline = isOfflineScan(context);
 -    boolean isolated = isIsolated(context);
 -    boolean localIterators = usesLocalIterators(context);
 -    boolean mockInstance = (null != instance && MockInstance.class.equals(instance.getClass()));
 -    Set<Pair<Text,Text>> fetchedColumns = getFetchedColumns(context);
 -    Authorizations auths = getScanAuthorizations(context);
 -    String principal = getPrincipal(context);
 -    String tokenClass = getTokenClass(context);
 -    byte[] tokenBytes = getToken(context);
 -    
 -    AuthenticationToken token;
 -    try {
 -       token = CredentialHelper.extractToken(tokenClass, tokenBytes);
 -    } catch (AccumuloSecurityException e) {
 -      throw new IOException(e);
 -    }
 -    
 -    List<IteratorSetting> iterators = getIterators(context);
 -    
 -    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 (isOfflineScan(context)) {
 -        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
 -          binnedRanges = binOfflineTable(context, tableName, ranges);
 -        }
 -      } else {
 -        String tableId = null;
 -        tl = getTabletLocator(context);
 -        // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
 -        tl.invalidateCache();
 -        while (!tl.binRanges(ranges, binnedRanges, new TCredentials(principal, tokenClass, ByteBuffer.wrap(tokenBytes), instance.getInstanceID())).isEmpty()) {
 -          if (!(instance instanceof MockInstance)) {
 -            if (tableId == null)
 -              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);
 -          }
 -          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);
 -    }
 -    
 -    ArrayList<InputSplit> splits = new ArrayList<InputSplit>(ranges.size());
 -    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(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(entry.getKey(), entry.getValue().toArray(new String[0])));
 -    
 -    for (InputSplit inputSplit : splits) {
 -      RangeInputSplit split = (RangeInputSplit) inputSplit;
 -
 -      split.setTable(tableName);
 -      split.setOffline(offline);
 -      split.setIsolatedScan(isolated);
 -      split.setUsesLocalIterators(localIterators);
 -      split.setMockInstance(mockInstance);
 -      split.setFetchedColumns(fetchedColumns);
 -      split.setPrincipal(principal);
 -      split.setToken(token);
 -      split.setInstanceName(instance.getInstanceName());
 -      split.setZooKeepers(instance.getZooKeepers());
 -      split.setAuths(auths);
 -      split.setIterators(iterators);
 -      split.setLogLevel(logLevel);
 -    }
 -    
 -    return splits;
 -  }
 -
 -  // ----------------------------------------------------------------------------------------------------
 -  // Everything below this line is deprecated and should go away in future versions
 -  // ----------------------------------------------------------------------------------------------------
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setScanIsolation(Job, boolean)} instead.
 -   */
 -  @Deprecated
 -  public static void setIsolated(Configuration conf, boolean enable) {
 -    InputConfigurator.setScanIsolation(CLASS, conf, enable);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setLocalIterators(Job, boolean)} instead.
 -   */
 -  @Deprecated
 -  public static void setLocalIterators(Configuration conf, boolean enable) {
 -    InputConfigurator.setLocalIterators(CLASS, conf, enable);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setConnectorInfo(Job, String, AuthenticationToken)}, {@link #setInputTableName(Job, String)}, and
 -   *             {@link #setScanAuthorizations(Job, Authorizations)} instead.
 -   */
 -  @Deprecated
 -  public static void setInputInfo(Configuration conf, String user, byte[] passwd, String table, Authorizations auths) {
 -    try {
 -      InputConfigurator.setConnectorInfo(CLASS, conf, user, new PasswordToken(passwd));
 -    } catch (AccumuloSecurityException e) {
 -      throw new RuntimeException(e);
 -    }
 -    InputConfigurator.setInputTableName(CLASS, conf, table);
 -    InputConfigurator.setScanAuthorizations(CLASS, conf, auths);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setZooKeeperInstance(Job, String, String)} instead.
 -   */
 -  @Deprecated
 -  public static void setZooKeeperInstance(Configuration conf, String instanceName, String zooKeepers) {
 -    InputConfigurator.setZooKeeperInstance(CLASS, conf, instanceName, zooKeepers);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setMockInstance(Job, String)} instead.
 -   */
 -  @Deprecated
 -  public static void setMockInstance(Configuration conf, String instanceName) {
 -    InputConfigurator.setMockInstance(CLASS, conf, instanceName);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setRanges(Job, Collection)} instead.
 -   */
 -  @Deprecated
 -  public static void setRanges(Configuration conf, Collection<Range> ranges) {
 -    InputConfigurator.setRanges(CLASS, conf, ranges);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setAutoAdjustRanges(Job, boolean)} instead.
 -   */
 -  @Deprecated
 -  public static void disableAutoAdjustRanges(Configuration conf) {
 -    InputConfigurator.setAutoAdjustRanges(CLASS, conf, false);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} to add the {@link VersioningIterator} instead.
 -   */
 -  @Deprecated
 -  public static void setMaxVersions(Configuration conf, int maxVersions) throws IOException {
 -    IteratorSetting vers = new IteratorSetting(1, "vers", VersioningIterator.class);
 -    try {
 -      VersioningIterator.setMaxVersions(vers, maxVersions);
 -    } catch (IllegalArgumentException e) {
 -      throw new IOException(e);
 -    }
 -    InputConfigurator.addIterator(CLASS, conf, vers);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setOfflineTableScan(Job, boolean)} instead.
 -   */
 -  @Deprecated
 -  public static void setScanOffline(Configuration conf, boolean scanOff) {
 -    InputConfigurator.setOfflineTableScan(CLASS, conf, scanOff);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #fetchColumns(Job, Collection)} instead.
 -   */
 -  @Deprecated
 -  public static void fetchColumns(Configuration conf, Collection<Pair<Text,Text>> columnFamilyColumnQualifierPairs) {
 -    InputConfigurator.fetchColumns(CLASS, conf, columnFamilyColumnQualifierPairs);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #setLogLevel(Job, Level)} instead.
 -   */
 -  @Deprecated
 -  public static void setLogLevel(Configuration conf, Level level) {
 -    InputConfigurator.setLogLevel(CLASS, conf, level);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} instead.
 -   */
 -  @Deprecated
 -  public static void addIterator(Configuration conf, IteratorSetting cfg) {
 -    InputConfigurator.addIterator(CLASS, conf, cfg);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #isIsolated(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static boolean isIsolated(Configuration conf) {
 -    return InputConfigurator.isIsolated(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #usesLocalIterators(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static boolean usesLocalIterators(Configuration conf) {
 -    return InputConfigurator.usesLocalIterators(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getPrincipal(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static String getPrincipal(Configuration conf) {
 -    return InputConfigurator.getPrincipal(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getToken(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static byte[] getToken(Configuration conf) {
 -    return InputConfigurator.getToken(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getInputTableName(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static String getTablename(Configuration conf) {
 -    return InputConfigurator.getInputTableName(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getScanAuthorizations(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static Authorizations getAuthorizations(Configuration conf) {
 -    return InputConfigurator.getScanAuthorizations(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getInstance(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static Instance getInstance(Configuration conf) {
 -    return InputConfigurator.getInstance(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getTabletLocator(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static TabletLocator getTabletLocator(Configuration conf) throws TableNotFoundException {
 -    return InputConfigurator.getTabletLocator(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getRanges(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static List<Range> getRanges(Configuration conf) throws IOException {
 -    return InputConfigurator.getRanges(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getFetchedColumns(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static Set<Pair<Text,Text>> getFetchedColumns(Configuration conf) {
 -    return InputConfigurator.getFetchedColumns(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getAutoAdjustRanges(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static boolean getAutoAdjustRanges(Configuration conf) {
 -    return InputConfigurator.getAutoAdjustRanges(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getLogLevel(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static Level getLogLevel(Configuration conf) {
 -    return InputConfigurator.getLogLevel(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #validateOptions(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static void validateOptions(Configuration conf) throws IOException {
 -    InputConfigurator.validateOptions(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #addIterator(Job, IteratorSetting)} to add the {@link VersioningIterator} instead.
 -   */
 -  @Deprecated
 -  protected static int getMaxVersions(Configuration conf) {
 -    // This is so convoluted, because the only reason to get the number of maxVersions is to construct the same type of IteratorSetting object we have to
 -    // deconstruct to get at this option in the first place, but to preserve correct behavior, this appears necessary.
 -    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
 -    for (IteratorSetting setting : iteratorSettings) {
 -      if ("vers".equals(setting.getName()) && 1 == setting.getPriority() && VersioningIterator.class.getName().equals(setting.getIteratorClass())) {
 -        if (setting.getOptions().containsKey("maxVersions"))
 -          return Integer.parseInt(setting.getOptions().get("maxVersions"));
 -        else
 -          return -1;
 -      }
 -    }
 -    return -1;
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #isOfflineScan(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static boolean isOfflineScan(Configuration conf) {
 -    return InputConfigurator.isOfflineScan(CLASS, conf);
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getIterators(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static List<AccumuloIterator> getIterators(Configuration conf) {
 -    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
 -    List<AccumuloIterator> deprecatedIterators = new ArrayList<AccumuloIterator>(iteratorSettings.size());
 -    for (IteratorSetting setting : iteratorSettings) {
 -      AccumuloIterator deprecatedIter = new AccumuloIterator(new String(setting.getPriority() + AccumuloIterator.FIELD_SEP + setting.getIteratorClass()
 -          + AccumuloIterator.FIELD_SEP + setting.getName()));
 -      deprecatedIterators.add(deprecatedIter);
 -    }
 -    return deprecatedIterators;
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link #getIterators(JobContext)} instead.
 -   */
 -  @Deprecated
 -  protected static List<AccumuloIteratorOption> getIteratorOptions(Configuration conf) {
 -    List<IteratorSetting> iteratorSettings = InputConfigurator.getIterators(CLASS, conf);
 -    List<AccumuloIteratorOption> deprecatedIteratorOptions = new ArrayList<AccumuloIteratorOption>(iteratorSettings.size());
 -    for (IteratorSetting setting : iteratorSettings) {
 -      for (Entry<String,String> opt : setting.getOptions().entrySet()) {
 -        String deprecatedOption;
 -        try {
 -          deprecatedOption = new String(setting.getName() + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getKey(), "UTF-8")
 -              + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getValue(), "UTF-8"));
 -        } catch (UnsupportedEncodingException e) {
 -          throw new RuntimeException(e);
 -        }
 -        deprecatedIteratorOptions.add(new AccumuloIteratorOption(deprecatedOption));
 -      }
 -    }
 -    return deprecatedIteratorOptions;
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link IteratorSetting} instead.
 -   */
 -  @Deprecated
 -  static class AccumuloIterator {
 -    
 -    private static final String FIELD_SEP = ":";
 -    
 -    private int priority;
 -    private String iteratorClass;
 -    private String iteratorName;
 -    
 -    public AccumuloIterator(int priority, String iteratorClass, String iteratorName) {
 -      this.priority = priority;
 -      this.iteratorClass = iteratorClass;
 -      this.iteratorName = iteratorName;
 -    }
 -    
 -    // Parses out a setting given an string supplied from an earlier toString() call
 -    public AccumuloIterator(String iteratorSetting) {
 -      // Parse the string to expand the iterator
 -      StringTokenizer tokenizer = new StringTokenizer(iteratorSetting, FIELD_SEP);
 -      priority = Integer.parseInt(tokenizer.nextToken());
 -      iteratorClass = tokenizer.nextToken();
 -      iteratorName = tokenizer.nextToken();
 -    }
 -    
 -    public int getPriority() {
 -      return priority;
 -    }
 -    
 -    public String getIteratorClass() {
 -      return iteratorClass;
 -    }
 -    
 -    public String getIteratorName() {
 -      return iteratorName;
 -    }
 -    
 -    @Override
 -    public String toString() {
 -      return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName);
 -    }
 -    
 -  }
 -  
 -  /**
 -   * @deprecated since 1.5.0; Use {@link IteratorSetting} instead.
 -   */
 -  @Deprecated
 -  static class AccumuloIteratorOption {
 -    private static final String FIELD_SEP = ":";
 -    
 -    private String iteratorName;
 -    private String key;
 -    private String value;
 -    
 -    public AccumuloIteratorOption(String iteratorName, String key, String value) {
 -      this.iteratorName = iteratorName;
 -      this.key = key;
 -      this.value = value;
 -    }
 -    
 -    // Parses out an option given a string supplied from an earlier toString() call
 -    public AccumuloIteratorOption(String iteratorOption) {
 -      StringTokenizer tokenizer = new StringTokenizer(iteratorOption, FIELD_SEP);
 -      this.iteratorName = tokenizer.nextToken();
 -      try {
 -        this.key = URLDecoder.decode(tokenizer.nextToken(), "UTF-8");
 -        this.value = URLDecoder.decode(tokenizer.nextToken(), "UTF-8");
 -      } catch (UnsupportedEncodingException e) {
 -        throw new RuntimeException(e);
 -      }
 -    }
 -    
 -    public String getIteratorName() {
 -      return iteratorName;
 -    }
 -    
 -    public String getKey() {
 -      return key;
 -    }
 -    
 -    public String getValue() {
 -      return value;
 -    }
 -    
 -    @Override
 -    public String toString() {
 -      try {
 -        return new String(iteratorName + FIELD_SEP + URLEncoder.encode(key, "UTF-8") + FIELD_SEP + URLEncoder.encode(value, "UTF-8"));
 -      } catch (UnsupportedEncodingException e) {
 -        throw new RuntimeException(e);
 -      }
 -    }
 -    
 -  }
 -  
 -  // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility
 -  static Configuration getConfiguration(JobContext context) {
 -    try {
 -      Class<?> c = InputFormatBase.class.getClassLoader().loadClass("org.apache.hadoop.mapreduce.JobContext");
 -      Method m = c.getMethod("getConfiguration");
 -      Object o = m.invoke(context, new Object[0]);
 -      return (Configuration) o;
 -    } catch (Exception e) {
 -      throw new RuntimeException(e);
 +      for (IteratorSetting iterator : iterators)
 +        scanner.addScanIterator(iterator);
      }
    }
 -  
  }